You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Yun Gao <yu...@aliyun.com.INVALID> on 2020/10/09 06:16:30 UTC

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

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 tasks finished, which causes some problems for bounded or mixed jobs:
Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5]. 
The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6]. 
Best,
Yun

[1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
[2] https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
[3] https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
[4] https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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

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

Posted by Khachatryan Roman <kh...@gmail.com>.
Hi Yun,

> b)  With unaligned checkpoint enabled, the slower cases might happen if
the downstream task processes very slowly.
I think UC will be the common case with multiple sources each with DoP > 1.
IIUC, waiting for EoP will be needed on each subtask each time one of it's
source subtask finishes.

> But since only the result partition part of the finished upstream need
wait to be processed, the other part of
> the execution graph could  still perform the unaligned checkpoint normally
Yes, but checkpoint completion notification will not be sent until all the
EOPs are processed.

> Declining the RPC-trigger checkpoint would indeed simplify the
implementation, but since currently by default the
> failed checkpoint would cause job failover, thus we might have some
concerns in directly decline the checkpoint.
Not all declines cause job failure, particularly
CHECKPOINT_DECLINED_TASK_NOT_READY doesn't.

> Thus another possible option might be let the upstream task to wait till
all the pending buffers in the result partition has been flushed before get
to finish.
This is what I meant by "postpone JM notification from source". Just
blocking the task thread wouldn't add much complexity, though I'm not sure
if it would cause any problems.

> do you think it would be ok for us to view it as an optimization and
postpone it to future versions ?
I think that's a good idea.

Regards,
Roman


On Mon, Jan 11, 2021 at 11:03 AM Yun Gao <yu...@aliyun.com> wrote:

>       Hi Roman,
>
>           Very thanks for the feedbacks !
>
>
>         > Probably it would be simpler to just decline the RPC-triggered
> checkpoint
>         > if not all inputs of this task are finished (with
> CHECKPOINT_DECLINED_TASK_NOT_READY).
>
>         > But I wonder how significantly this waiting for EoP from every
> input will delay performing the first checkpoint
>         > by B after becoming a new source. This may in turn impact
> exactly-once sinks and incremental checkpoints.
>         > Maybe a better option would be to postpone JM notification from
> source until it's EoP is consumed?
>
>        I also agree with that there would indeed be possible cases that
> the checkpoint get slower since it could not skip
>        the data in  the result partition of the finished upstream task:
>             a) For aligned checkpoint, the cases would not happen since
> the downstream tasks would always need to
>                 process the buffers in order.
>            b)  With unaligned checkpoint enabled, the slower cases might
> happen if the downstream task processes very
>                 slowly.
>
>        But since only the result partition part of the finished upstream
> need wait to be processed, the other part of
>        the execution graph could  still perform the unaligned checkpoint
> normally, I think the average delay caused would
>        be much lower than the completely aligned checkpoint, but there
> would still be extremely bad cases that
>        the delay is long.
>
>        Declining the RPC-trigger checkpoint would indeed simplify the
> implementation, but since currently by default the
>        failed checkpoint would cause job failover, thus we might have some
> concerns in directly decline the checkpoint.
>        For postpone the notification the JM notification, since current JM
> should not be able to know if the task has
>        received all the EndOfPartition from the upstream tasks, we might
> need to introduce new RPC for notifying the
>        state and since the triggering is not atomic, we may also met with
> some  synchronization issues between JM and TM,
>        which would introduce some complexity.
>
>       Thus another possible option might be let the upstream task to wait
> till all the pending buffers in the result partition has
>       been flushed before get to finish. We could only do the wait for the
> PipelineResultPartition so it won't affect the batch
>       jobs. With the waiting the unaligned checkpoint could continue to
> trigger the upstream task and skip the buffers in
>       the result partition. Since the result partition state would be kept
> within the operator state of the last operator, after failover
>       we would found that the last operator has an non-empty state and we
> would restart the tasks containing this operator to
>       resend the snapshotted buffers. Of course this would also introduce
> some complexity, and since the probability of long delay
>       would be lower than the completely aligned case, do you think it
> would be ok for us to view it as an optimization and
>       postpone it to future versions ?
>
>      Best,
>      Yun
>
>
>
> ------------------------------------------------------------------
> From:Khachatryan Roman <kh...@gmail.com>
> Send Time:2021 Jan. 11 (Mon.) 05:46
> To:Yun Gao <yu...@aliyun.com>
> Cc:Arvid Heise <ar...@ververica.com>; dev <de...@flink.apache.org>; user <
> user@flink.apache.org>
> Subject:Re: Re: [DISCUSS] 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" to trigger checkpoint via RPC. For task B,
> if it received checkpoint trigger, it would know that all its precedant
> tasks
> > are finished, then it would wait till all the InputChannel received
> EndOfPartition from the network (namely inputChannel.onBuffer() 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.
>
> Probably it would be simpler to just decline the RPC-triggered checkpoint
> if not all inputs of this task are finished (with
> CHECKPOINT_DECLINED_TASK_NOT_READY).
>
> But I wonder how significantly this waiting for EoP from every input will
> delay performing the first checkpoint by B after becoming a new source.
> This may in turn impact exactly-once sinks and incremental checkpoints.
> Maybe a better option would be to postpone JM notification from source
> until it's EoP is consumed?
>
> Regards,
> Roman
>
>
>

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

Posted by Khachatryan Roman <kh...@gmail.com>.
Hi Yun,

> b)  With unaligned checkpoint enabled, the slower cases might happen if
the downstream task processes very slowly.
I think UC will be the common case with multiple sources each with DoP > 1.
IIUC, waiting for EoP will be needed on each subtask each time one of it's
source subtask finishes.

> But since only the result partition part of the finished upstream need
wait to be processed, the other part of
> the execution graph could  still perform the unaligned checkpoint normally
Yes, but checkpoint completion notification will not be sent until all the
EOPs are processed.

> Declining the RPC-trigger checkpoint would indeed simplify the
implementation, but since currently by default the
> failed checkpoint would cause job failover, thus we might have some
concerns in directly decline the checkpoint.
Not all declines cause job failure, particularly
CHECKPOINT_DECLINED_TASK_NOT_READY doesn't.

> Thus another possible option might be let the upstream task to wait till
all the pending buffers in the result partition has been flushed before get
to finish.
This is what I meant by "postpone JM notification from source". Just
blocking the task thread wouldn't add much complexity, though I'm not sure
if it would cause any problems.

> do you think it would be ok for us to view it as an optimization and
postpone it to future versions ?
I think that's a good idea.

Regards,
Roman


On Mon, Jan 11, 2021 at 11:03 AM Yun Gao <yu...@aliyun.com> wrote:

>       Hi Roman,
>
>           Very thanks for the feedbacks !
>
>
>         > Probably it would be simpler to just decline the RPC-triggered
> checkpoint
>         > if not all inputs of this task are finished (with
> CHECKPOINT_DECLINED_TASK_NOT_READY).
>
>         > But I wonder how significantly this waiting for EoP from every
> input will delay performing the first checkpoint
>         > by B after becoming a new source. This may in turn impact
> exactly-once sinks and incremental checkpoints.
>         > Maybe a better option would be to postpone JM notification from
> source until it's EoP is consumed?
>
>        I also agree with that there would indeed be possible cases that
> the checkpoint get slower since it could not skip
>        the data in  the result partition of the finished upstream task:
>             a) For aligned checkpoint, the cases would not happen since
> the downstream tasks would always need to
>                 process the buffers in order.
>            b)  With unaligned checkpoint enabled, the slower cases might
> happen if the downstream task processes very
>                 slowly.
>
>        But since only the result partition part of the finished upstream
> need wait to be processed, the other part of
>        the execution graph could  still perform the unaligned checkpoint
> normally, I think the average delay caused would
>        be much lower than the completely aligned checkpoint, but there
> would still be extremely bad cases that
>        the delay is long.
>
>        Declining the RPC-trigger checkpoint would indeed simplify the
> implementation, but since currently by default the
>        failed checkpoint would cause job failover, thus we might have some
> concerns in directly decline the checkpoint.
>        For postpone the notification the JM notification, since current JM
> should not be able to know if the task has
>        received all the EndOfPartition from the upstream tasks, we might
> need to introduce new RPC for notifying the
>        state and since the triggering is not atomic, we may also met with
> some  synchronization issues between JM and TM,
>        which would introduce some complexity.
>
>       Thus another possible option might be let the upstream task to wait
> till all the pending buffers in the result partition has
>       been flushed before get to finish. We could only do the wait for the
> PipelineResultPartition so it won't affect the batch
>       jobs. With the waiting the unaligned checkpoint could continue to
> trigger the upstream task and skip the buffers in
>       the result partition. Since the result partition state would be kept
> within the operator state of the last operator, after failover
>       we would found that the last operator has an non-empty state and we
> would restart the tasks containing this operator to
>       resend the snapshotted buffers. Of course this would also introduce
> some complexity, and since the probability of long delay
>       would be lower than the completely aligned case, do you think it
> would be ok for us to view it as an optimization and
>       postpone it to future versions ?
>
>      Best,
>      Yun
>
>
>
> ------------------------------------------------------------------
> From:Khachatryan Roman <kh...@gmail.com>
> Send Time:2021 Jan. 11 (Mon.) 05:46
> To:Yun Gao <yu...@aliyun.com>
> Cc:Arvid Heise <ar...@ververica.com>; dev <de...@flink.apache.org>; user <
> user@flink.apache.org>
> Subject:Re: Re: [DISCUSS] 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" to trigger checkpoint via RPC. For task B,
> if it received checkpoint trigger, it would know that all its precedant
> tasks
> > are finished, then it would wait till all the InputChannel received
> EndOfPartition from the network (namely inputChannel.onBuffer() 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.
>
> Probably it would be simpler to just decline the RPC-triggered checkpoint
> if not all inputs of this task are finished (with
> CHECKPOINT_DECLINED_TASK_NOT_READY).
>
> But I wonder how significantly this waiting for EoP from every input will
> delay performing the first checkpoint by B after becoming a new source.
> This may in turn impact exactly-once sinks and incremental checkpoints.
> Maybe a better option would be to postpone JM notification from source
> until it's EoP is consumed?
>
> Regards,
> Roman
>
>
>

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

Posted by Dawid Wysakowicz <dw...@apache.org>.
+1 to what Arvid said.

I am also thinking we could even consider dropping the dispose method
straightaway to make the need for migration obvious. I'd make that
decision during the implementation/on the PR though, once we verify if
the deprecation option works.

Best,

Dawid

On 10/06/2021 09:37, Arvid Heise wrote:
> The whole operator API is only for advanced users and is not marked
> Public(Evolving). Users have to accept that things change and we have to
> use that freedom that we don't have in many other parts of the system.
>
> The change needs to be very clear in the change notes though. I also don't
> expect many users to be affected even if they use operator API and if so,
> the change will be quite small. Note that we should still include the
> change in the compatibility section of the FLIP even though the API is
> non-public.
>
> On Thu, Jun 10, 2021 at 8:08 AM Yun Gao <yu...@aliyun.com.invalid>
> wrote:
>
>> Hi all,
>>
>> Very thanks for the warm discussions!
>>
>> Regarding the change in the operator lifecycle, I also agree with adding
>> the flush/drain/stopAndFlush/finish method. For the duplication between
>> this
>> method and `endInput` for one input operator, with some offline disucssion
>> with
>> Dawid now I also think we might instead consider dropping the
>> BoundedOneInput
>> in the future and thus it should not be an issue.
>>
>> But for the compatibility problem I still have one concern: for the new
>> lifecycle
>> we would remove `dispose()` and call `close()` for both normal finish and
>> failover, if some
>> users use customized operator that only overrides `close()` method, with
>> the new lifecycle
>> it would still compile, but might have wrong result if there are failover.
>> Perhaps the SQL
>> mini-batch operator (AbstractMapBundleOperator) is an example: it finish
>> the last mini-batch
>> in the close() method, if the method also get called in failover, the last
>> mini-batch would get emitted,
>> and these data would also replayed after failover, which would cause data
>> repeatation. But considering
>> there might be not too much users write customized operators, would this
>> be a blocker ?
>>
>> Regarding the name of the method, I would also tend to simpler name,
>> perhaps finish (should be more easier
>> for user to understand) or drain (since the method should only be called
>> on termination and stop-with-savepoint
>> --drain, the concept would be more consistent).
>>
>> Regarding the changes in the UDF, I also agree with that to avoid the
>> output type issue,  we may only
>> add flush/drain/stopAndFlush/finish method to sink function for now. In
>> the future if there
>> are more requirments we may further consider add the method to the
>> low-level UDF like
>> ProcessFunction / CoProcessFunction as an advanced feature.
>>
>> Best,
>> Yun
>>
>>
>>
>>
>>
>>
>>
>>
>>  ------------------Original Mail ------------------
>> Sender:Arvid Heise <ar...@apache.org>
>> Send Date:Thu Jun 10 01:08:36 2021
>> Recipients:dev <de...@flink.apache.org>
>> CC:Yun Gao <yu...@aliyun.com>, Till Rohrmann <tr...@apache.org>
>> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>> Hi Piot,
>>
>>
>>
>> I'm fine with just doing it on the Sink. My responses were focused on the
>>
>> API (the how) not on the concept (the if). Just keep the methods on the
>>
>> different places in sync, such that it is easy to introduce a common
>>
>> interface later.
>>
>>
>>
>> Re name: drain is not a reinvention as it's used quite often throughout
>>
>> Flink (e.g., Mailbox, stopWithSavepoint with drain flag). flush has no link
>>
>> to life-cycles: you usually do it arbitrarily often.
>>
>> I like `finish` very much as it relates to JobStatus FINISHED, has a clear
>>
>> tie to life-cycles, and is crisp.
>>
>> I also thought about `terminate` but I'd clearly favor `finish` as the
>>
>> verbs cannot be exchanged in the following: the task may terminate its
>>
>> operators but the operators should finish their thing first.
>>
>>
>>
>> On Wed, Jun 9, 2021 at 6:48 PM Piotr Nowojski  wrote:
>>
>>
>>
>>> Hi,
>>> Arvid: What's the problem with providing `void flush()`/`void drain()`
>> only
>>
>>> in the `SinkFunction`? It would avoid the problem of typing. Why would
>> one
>>
>>> need to have it in the other `Rich***Function`s? For `flush()` to make
>>> sense, the entity which has this method, would need to buffer some data.
>> I
>>
>>> don't see this to be reasonable in anything but `SinkFunction`,
>>> `ProcessFunction` and operators.
>>> Re `flush()` vs `drain()`. Frankly, I wouldn't understand what `drain()`
>> is
>>
>>> all about without reading the java-doc, and afterwards, I would have an
>>> impression that someone wanted to reinvent a wheel :) `flush()` is kind
>> of
>>
>>> an industry standard for things like that. Furthermore I don't think
>>> `drain()` solves Till's concern (drain != stop + flush). `stopAndFlush()`
>>> would be better in this regard, but it also doesn't feel right. Maybe
>>> `finish()`?
>>> Piotrek
>>> śr., 9 cze 2021 o 11:51 Arvid Heise  napisał(a):
>>>> Hi Dawid,
>>>> I see your point. I'd probably add drain only to Rich*Function where we
>>>> have the type bounds. Then we still need your Flushable interface in
>>>> Rich*Function<..., T> to call it efficiently but we at least avoid
>> weird
>>
>>>> type combinations. I'll have a rethink later.
>>>> The proper solution is probably to add  to RichFunction and use Void
>>>> for RichSinkFunction but I'll have to understand the implications
>> first.
>>
>>>> On Wed, Jun 9, 2021 at 11:37 AM Dawid Wysakowicz
>>>>
>>>> wrote:
>>>>> Hey,
>>>>> @Arvid The problem with adding the "drain/flush/stopProcessing" method
>>> to
>>>>> RichFunction is that it is not typed with the output type. At the same
>>> time
>>>>> we would most likely need a way to emit records from the method.
>> That's
>>
>>>>> originally thought about adding a typed interface which honestly I
>> don't
>>
>>>>> like that much either.
>>>>> On the UDF level we do not need to deprecate anything as you said. The
>>>>> close there already works as dispose on the Operator level. What we
>> are
>>
>>>>> suggesting is to unify that on the Operator level and deprecate the
>>> dispose
>>>>> there. @Yun I think we can already do that. We can either try to
>> handle
>>
>>>>> exceptions from the close in the case of a failure or just break it as
>>> it
>>>>> is a low-level, mostly internal API as Arvid said and also the
>> migration
>>
>>>>> would be really easy there.
>>>>> @Till @Arvid I am open for suggestions about the naming. I like the
>>>>> "drain" method.
>>>>> For now I'd go with @Piotr's proposal to add the "drain" method only
>> to
>>
>>>>> the SinkFunction. We think they are not immediately necessary for any
>> of
>>
>>>>> the other UDFs.
>>>>> Best,
>>>>> Dawid
>>>>> On 09/06/2021 11:20, Arvid Heise wrote:
>>>>> I have not followed the complete discussion and can't comment on the
>>>>> concepts. However, I have some ideas on the API changes:
>>>>> 1. If it's about adding additional life-cycle methods to UDFs, we
>> should
>>
>>>>> add the flush/endOfInput to RichFunction as this is the current way to
>>>>> define it. At this point, I don't see the need to add/change anything
>>> for
>>>>> UDFs. Since RichFunction does not have a dispose, do we even need to
>>>>> deprecate anything on UDF level? This would avoid having a new
>> interface
>>
>>>>> Flushable altogether (of which I'm not a big fan, see Piot's mail)
>>>>> 2. Further, I'd like to propose drain instead of flush as it would
>> more
>>
>>>>> align with the current nomenclature and makes the intent more obvious.
>>>>> However, that only works if there is no clash, so please double-check.
>>>>> 3. About changing methods on Operators: I'd say go ahead. It's
>>>>> experimental and not too hard to adjust on the user side. I also like
>>> the
>>>>> idea of beefing up ProcessFunction as a full replacement to custom
>>>>> operators but I'd keep that effort separate.
>>>>> On Wed, Jun 9, 2021 at 9:38 AM Till Rohrmann
>>>>> wrote:
>>>>>> Thanks for the lively discussion everyone. I have to admit that I am
>>> not
>>>>>> really convinced that we should call the interface Flushable and the
>>>>>> method
>>>>>> flush. The problem is that this method should in the first place tell
>>> the
>>>>>> operator that it should stop processing and flush all buffered data.
>>> The
>>>>>> method "flush" alone does not convey this contract very well. Maybe a
>>>>>> more
>>>>>> explicit name like stopProcessingAndFlush (maybe only stopProcessing)
>>>>>> would
>>>>>> be better. Moreover, from the OutputStream.flush method, I would
>> expect
>>
>>>>>> that I can call this method multiple times w/o changing the state of
>>> the
>>>>>> stream. This is not the case here.
>>>>>> Given that the stop processing and flush all results is such an
>>> essential
>>>>>> lifecycle method of an operator/UDF, I am not sure whether we should
>>>>>> offer
>>>>>> it as an optional interface users can implement. The problem I see is
>>>>>> that
>>>>>> users are not aware of it when writing their own operators/UDFs.
>> Making
>>
>>>>>> it
>>>>>> part of the actual interfaces makes it more explicit and easier to
>>>>>> discover. Maybe there is a way of adding it together with a default
>>>>>> implementation, deprecating the other methods, and then eventually
>>>>>> removing
>>>>>> the old methods. The last step will break APIs, though :-(
>>>>>> Cheers,
>>>>>> Till
>>>>>> On Tue, Jun 8, 2021 at 6:27 PM Piotr Nowojski
>>>>>> wrote:
>>>>>>> Hi,
>>>>>>> Thanks for resuming this discussion. I think +1 for the proposal of
>>>>>>> dropping (deprecating) `dispose()`, and adding `flush()` to the
>>>>>>> `StreamOperator`/udfs. Semantically it would be more like new
>>>>>> `close()` is
>>>>>>> an equivalent of old `dispose()`. Old `close()` is an equivalent of
>>> new
>>>>>>> `flush() + close()`. I think it provides a relatively painless
>>>>>> migration
>>>>>>> path (could we write down this migration?).
>>>>>>> However I have some doubts about the Flushable interface. First of
>>>>>> all,
>>>>>>> it wouldn't work for sinks - sinks have no output. Secondly, I
>> don't
>>
>>>>>> like
>>>>>>> that it opens a possibility for problems like this (incompatible
>>> output
>>>>>>> types):
>>>>>>> ```
>>>>>>> public class MyMap implements MapFunction,
>>>>>> Flushable
>>>>>>> { ...}
>>>>>>> ```
>>>>>>> Also after a quick offline discussion with Dawid, I'm not sure
>>> anymore
>>>>>> to
>>>>>>> which UDFs it actually makes sense to add `flush`, as most of them
>>>>>>> shouldn't buffer any data. Apart from Sinks, it's usually an
>> operator
>>
>>>>>> that
>>>>>>> is buffering the data (that holds true for AsyncFunction,
>>>>>> ReduceFunction,
>>>>>>> AggregateFunction, MapFunction, FilterFunction, ...). For those
>>>>>> functions
>>>>>>> it's difficult to buffer any data, as they have no means to control
>>>>>> when to
>>>>>>> emit this data. One notable exception might be (Co)ProcessFunction,
>>> as
>>>>>> it
>>>>>>> can register timers on it's own. In that case I would propose to do
>>> the
>>>>>>> following thing:
>>>>>>> 1. Add `flush() {}` to `Sink` function (FLIP-143 Writer interface
>>>>>> already
>>>>>>> has flush capabilities)
>>>>>>> 2. Maybe add `flush(Collector)` to `(Co)ProcessFunction`, but
>>> maybe
>>>>>> we
>>>>>>> can postpone it
>>>>>>> 3. Leave other functions alone.
>>>>>>> After all, we could add `flush()` to other functions in the future
>> if
>>
>>>>>> we
>>>>>>> really find a good motivating example to do so.
>>>>>>> About 2. Dawid is pitching an idea to convert `ProcessFunction`
>> into
>>
>>> a
>>>>>>> proper `Public` API that would replace StreamOperator. We could
>>> change
>>>>>>> `StreamOperator` to be purely `@Internal` class/interface, and add
>>> the
>>>>>>> missing functionality to the `ProcessFunction` (InputSelectable,
>>>>>>> BoundedInput, MailboxExecutor). With this, adding `flush()` to
>>>>>>> `ProcessFunction` would make a lot of sense. But maybe that should
>>> be a
>>>>>>> story for another day?
>>>>>>> Best,
>>>>>>> Piotrek
>>>>>>> pt., 4 cze 2021 o 10:36 Yun Gao  napisał(a):
>>>>>>>> Hi all,
>>>>>>>> Very thanks @Dawid for resuming the discussion and very thanks
>> @Till
>>
>>>>>> for
>>>>>>>> the summary ! (and very sorry for I missed the mail and do not
>>>>>> response
>>>>>>>> in time...)
>>>>>>>> I also agree with that we could consider the global commits latter
>>>>>>>> separately after we have addressed the final checkpoints, and also
>>>>>> other
>>>>>>>> points as Till summarized.
>>>>>>>> Currently the only case that have used the cascade commit is the
>>> Table
>>>>>>>> FileSystem and Hive connectors. I checked the code and found
>>>>>> currently they
>>>>>>>> will commit the
>>>>>>>> last piece of data directly in endOfInput(). Although this might
>>> emit
>>>>>>>> repeat records if there are failover during job finishing, it
>> avoids
>>
>>>>>>>> emitting the records in the
>>>>>>>> notifyCheckpointComplete() after endOfInput(), thus the
>> modification
>>
>>>>>> to
>>>>>>>> the operator lifecycle in final checkpoints would cause
>>> compatibility
>>>>>>>> problem for these connectors,
>>>>>>>> thus we do not need to modify them at the first place.
>>>>>>>> 2. Regarding the operator lifecycle, I also agree with the
>> proposed
>>
>>>>>>>> changes. To sum up, I think the operator lifecycle would become
>>>>>>>> endOfInput(1)
>>>>>>>> ...
>>>>>>>> endOfInput(n)
>>>>>>>> flush() --> call UDF's flush method
>>>>>>>> if some operator requires final checkpoints
>>>>>>>> snapshotState()
>>>>>>>> notifyCheckpointComplete()
>>>>>>>> end if
>>>>>>>> close() --> call UDF's close method
>>>>>>>> Since currently the close() is only called in normal finish and
>>>>>> dispose()
>>>>>>>> will be called in both failover and normal case, for
>> compatibility,
>>
>>> I
>>>>>> think
>>>>>>>> we may
>>>>>>>> have to postpone the change to a single close() method to version
>>> 2.0
>>>>>> ?
>>>>>>>> 3. Regarding the name and position of flush() method, I also agree
>>>>>> with
>>>>>>>> that we will need a separate method to mark the termination of the
>>>>>> whole
>>>>>>>> stream for
>>>>>>>> multiple-input streams. Would it be also ok if we have some
>>>>>> modification
>>>>>>>> to the current BoundedXXInput interfaces to
>>>>>>>> interface BoundedInput {
>>>>>>>> void endInput() // marks the end of the whole streams, as
>>> flush()
>>>>>>>> does.
>>>>>>>> }
>>>>>>>> @deprecated // In the future we could remove this interface
>>>>>>>> interface BoundedOneInput extends BoundedInput {}
>>>>>>>> interface BoundedMultiInput extends BoundedInput {
>>>>>>>> void endInput(int i);
>>>>>>>> default void endInput() {} // For compatibility
>>>>>>>> }
>>>>>>>> If operator/UDF does not care about the end of a single input,
>> then
>>
>>> it
>>>>>>>> could directly implement the BoundedInput interface. The possible
>>>>>>>> benefit to me is that we might be able to keep only one concept
>> for
>>
>>>>>>>> marking the end of stream, especially for the operators with only
>>>>>>>> one input.
>>>>>>>> Very thanks for all the deep insights and discussions!
>>>>>>>> Best,
>>>>>>>> Yun
>>>>>>>> ------------------------------------------------------------------
>>>>>>>> From:Dawid Wysakowicz
>>>>>>>> Send Time:2021 Jun. 3 (Thu.) 21:21
>>>>>>>> To:dev ; Till Rohrmann
>>>> ;
>>>>>> Yun
>>>>>>>> Gao
>>>>>>>> Cc:Piotr Nowojski ; Guowei Ma <
>>>>>> guowei.mgw@gmail.com>;
>>>>>>>> Stephan Ewen
>>>>>>>> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
>>>>>> Finished
>>>>>>>> Hi all,
>>>>>>>> Thanks for the very insightful discussion. I'd like to revive the
>>>>>> effort
>>>>>>>> of FLIP-147. First of all, from my side I'd like to say that I am
>>>>>> really
>>>>>>>> interested in helping that happen in the upcoming 1.14 release.
>>>>>>>> I agree with Till that the final checkpoints and global commits
>> are
>>
>>>>>>>> mostly orthogonal. Similarly as Till, I'd suggest to first focus
>> on
>>
>>>>>> the
>>>>>>>> final checkpoints, while just keeping in mind we should not make
>>>>>>>> assumptions that would make it impossible to implement the global
>>>>>> commits.
>>>>>>>> So far I do not see such risk from the discussion.
>>>>>>>> Going back to the final checkpoints issue. I think the only
>>>>>> outstanding
>>>>>>>> issue is which methods we want to use for flushing/closing both
>>>>>> operators
>>>>>>>> and UDFs just before performing the final checkpoint. As pointed
>> out
>>
>>>>>> to me
>>>>>>>> by Piotr, I am mentioning UDFs here as well, because we need a way
>>> for
>>>>>>>> users using the Public API to benefit from the final checkpoint
>>> (bear
>>>>>> in
>>>>>>>> mind that e.g. TwoPhaseCommitSinkFunction which is implemented by
>>> our
>>>>>> Kafka
>>>>>>>> sink operates on the UDF level). Right now RichFunction has no
>>> method
>>>>>> which
>>>>>>>> could be called just before the final checkpoint that would say
>>>>>> "flush" all
>>>>>>>> intermediate state now and prepare for the final checkpoint. I'd
>>>>>> suggest
>>>>>>>> introducing an additional interface e.g. (name to be determined)
>>>>>>>> interface Flushable {
>>>>>>>> void flush(Collector out)
>>>>>>>> }
>>>>>>>> Additionally we would need to introduce a similar method on the
>>>>>>>> StreamOperator level. Currently we have two methods that are
>> called
>>
>>>>>> at the
>>>>>>>> end of operator lifecycle:
>>>>>>>> -
>>>>>>>> - close
>>>>>>>> - dispose
>>>>>>>> The usage of the two methods is a bit confusing. Dispose is
>>>>>> responsible
>>>>>>>> for closing all open resources and is supposed to be called in
>> case
>>
>>>>>> of a
>>>>>>>> failure. On the other hand the close is a combination of a
>>>>>> non-existent
>>>>>>>> "flush" method we need and dispose for closing resources in case
>> of
>>
>>> a
>>>>>>>> successful run. I'd suggest to clear it a bit. We would introduce
>> a
>>
>>>>>> proper
>>>>>>>> "flush" method which would be called in case of a successful
>>>>>> finishing of
>>>>>>>> an operator. Moreover we would make "close" deal only with closing
>>>>>> any open
>>>>>>>> resources, basically taking over the role of the dispose, which we
>>>>>> would
>>>>>>>> deprecate.
>>>>>>>> Lastly, I'd like to say why I think it is better introduce a new
>>>>>> "flush"
>>>>>>>> method instead of using the "endInput" method of
>>>>>> BoundedOne/MultiInput.
>>>>>>>> That method is called per input, which means each operator would
>>> need
>>>>>> to
>>>>>>>> keep track of which inputs were already closed internally and
>> react
>>
>>>>>>>> differently if all of the inputs were closed. With an explicit
>>> "flush"
>>>>>>>> method we do not have such a problem as the input bookkeeping
>>> happens
>>>>>> on
>>>>>>>> the StreamTask level.
>>>>>>>> Let me know what you think. I'd sync with Yun Gao and if there are
>>> no
>>>>>>>> objections we will extend the FLIP page with necessary changes.
>>>>>>>> Best,
>>>>>>>> Dawid
>>


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

Posted by Arvid Heise <ar...@apache.org>.
The whole operator API is only for advanced users and is not marked
Public(Evolving). Users have to accept that things change and we have to
use that freedom that we don't have in many other parts of the system.

The change needs to be very clear in the change notes though. I also don't
expect many users to be affected even if they use operator API and if so,
the change will be quite small. Note that we should still include the
change in the compatibility section of the FLIP even though the API is
non-public.

On Thu, Jun 10, 2021 at 8:08 AM Yun Gao <yu...@aliyun.com.invalid>
wrote:

> Hi all,
>
> Very thanks for the warm discussions!
>
> Regarding the change in the operator lifecycle, I also agree with adding
> the flush/drain/stopAndFlush/finish method. For the duplication between
> this
> method and `endInput` for one input operator, with some offline disucssion
> with
> Dawid now I also think we might instead consider dropping the
> BoundedOneInput
> in the future and thus it should not be an issue.
>
> But for the compatibility problem I still have one concern: for the new
> lifecycle
> we would remove `dispose()` and call `close()` for both normal finish and
> failover, if some
> users use customized operator that only overrides `close()` method, with
> the new lifecycle
> it would still compile, but might have wrong result if there are failover.
> Perhaps the SQL
> mini-batch operator (AbstractMapBundleOperator) is an example: it finish
> the last mini-batch
> in the close() method, if the method also get called in failover, the last
> mini-batch would get emitted,
> and these data would also replayed after failover, which would cause data
> repeatation. But considering
> there might be not too much users write customized operators, would this
> be a blocker ?
>
> Regarding the name of the method, I would also tend to simpler name,
> perhaps finish (should be more easier
> for user to understand) or drain (since the method should only be called
> on termination and stop-with-savepoint
> --drain, the concept would be more consistent).
>
> Regarding the changes in the UDF, I also agree with that to avoid the
> output type issue,  we may only
> add flush/drain/stopAndFlush/finish method to sink function for now. In
> the future if there
> are more requirments we may further consider add the method to the
> low-level UDF like
> ProcessFunction / CoProcessFunction as an advanced feature.
>
> Best,
> Yun
>
>
>
>
>
>
>
>
>  ------------------Original Mail ------------------
> Sender:Arvid Heise <ar...@apache.org>
> Send Date:Thu Jun 10 01:08:36 2021
> Recipients:dev <de...@flink.apache.org>
> CC:Yun Gao <yu...@aliyun.com>, Till Rohrmann <tr...@apache.org>
> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
> Hi Piot,
>
>
>
> I'm fine with just doing it on the Sink. My responses were focused on the
>
> API (the how) not on the concept (the if). Just keep the methods on the
>
> different places in sync, such that it is easy to introduce a common
>
> interface later.
>
>
>
> Re name: drain is not a reinvention as it's used quite often throughout
>
> Flink (e.g., Mailbox, stopWithSavepoint with drain flag). flush has no link
>
> to life-cycles: you usually do it arbitrarily often.
>
> I like `finish` very much as it relates to JobStatus FINISHED, has a clear
>
> tie to life-cycles, and is crisp.
>
> I also thought about `terminate` but I'd clearly favor `finish` as the
>
> verbs cannot be exchanged in the following: the task may terminate its
>
> operators but the operators should finish their thing first.
>
>
>
> On Wed, Jun 9, 2021 at 6:48 PM Piotr Nowojski  wrote:
>
>
>
> > Hi,
>
> >
>
> > Arvid: What's the problem with providing `void flush()`/`void drain()`
> only
>
> > in the `SinkFunction`? It would avoid the problem of typing. Why would
> one
>
> > need to have it in the other `Rich***Function`s? For `flush()` to make
>
> > sense, the entity which has this method, would need to buffer some data.
> I
>
> > don't see this to be reasonable in anything but `SinkFunction`,
>
> > `ProcessFunction` and operators.
>
> >
>
> > Re `flush()` vs `drain()`. Frankly, I wouldn't understand what `drain()`
> is
>
> > all about without reading the java-doc, and afterwards, I would have an
>
> > impression that someone wanted to reinvent a wheel :) `flush()` is kind
> of
>
> > an industry standard for things like that. Furthermore I don't think
>
> > `drain()` solves Till's concern (drain != stop + flush). `stopAndFlush()`
>
> > would be better in this regard, but it also doesn't feel right. Maybe
>
> > `finish()`?
>
> >
>
> > Piotrek
>
> >
>
> > śr., 9 cze 2021 o 11:51 Arvid Heise  napisał(a):
>
> >
>
> > > Hi Dawid,
>
> > >
>
> > > I see your point. I'd probably add drain only to Rich*Function where we
>
> > > have the type bounds. Then we still need your Flushable interface in
>
> > > Rich*Function<..., T> to call it efficiently but we at least avoid
> weird
>
> > > type combinations. I'll have a rethink later.
>
> > >
>
> > > The proper solution is probably to add  to RichFunction and use Void
>
> > > for RichSinkFunction but I'll have to understand the implications
> first.
>
> > >
>
> > > On Wed, Jun 9, 2021 at 11:37 AM Dawid Wysakowicz
> > >
>
> > > wrote:
>
> > >
>
> > >> Hey,
>
> > >>
>
> > >> @Arvid The problem with adding the "drain/flush/stopProcessing" method
>
> > to
>
> > >> RichFunction is that it is not typed with the output type. At the same
>
> > time
>
> > >> we would most likely need a way to emit records from the method.
> That's
>
> > >> originally thought about adding a typed interface which honestly I
> don't
>
> > >> like that much either.
>
> > >>
>
> > >> On the UDF level we do not need to deprecate anything as you said. The
>
> > >> close there already works as dispose on the Operator level. What we
> are
>
> > >> suggesting is to unify that on the Operator level and deprecate the
>
> > dispose
>
> > >> there. @Yun I think we can already do that. We can either try to
> handle
>
> > >> exceptions from the close in the case of a failure or just break it as
>
> > it
>
> > >> is a low-level, mostly internal API as Arvid said and also the
> migration
>
> > >> would be really easy there.
>
> > >>
>
> > >> @Till @Arvid I am open for suggestions about the naming. I like the
>
> > >> "drain" method.
>
> > >>
>
> > >> For now I'd go with @Piotr's proposal to add the "drain" method only
> to
>
> > >> the SinkFunction. We think they are not immediately necessary for any
> of
>
> > >> the other UDFs.
>
> > >>
>
> > >> Best,
>
> > >>
>
> > >> Dawid
>
> > >> On 09/06/2021 11:20, Arvid Heise wrote:
>
> > >>
>
> > >> I have not followed the complete discussion and can't comment on the
>
> > >> concepts. However, I have some ideas on the API changes:
>
> > >>
>
> > >> 1. If it's about adding additional life-cycle methods to UDFs, we
> should
>
> > >> add the flush/endOfInput to RichFunction as this is the current way to
>
> > >> define it. At this point, I don't see the need to add/change anything
>
> > for
>
> > >> UDFs. Since RichFunction does not have a dispose, do we even need to
>
> > >> deprecate anything on UDF level? This would avoid having a new
> interface
>
> > >> Flushable altogether (of which I'm not a big fan, see Piot's mail)
>
> > >>
>
> > >> 2. Further, I'd like to propose drain instead of flush as it would
> more
>
> > >> align with the current nomenclature and makes the intent more obvious.
>
> > >> However, that only works if there is no clash, so please double-check.
>
> > >>
>
> > >> 3. About changing methods on Operators: I'd say go ahead. It's
>
> > >> experimental and not too hard to adjust on the user side. I also like
>
> > the
>
> > >> idea of beefing up ProcessFunction as a full replacement to custom
>
> > >> operators but I'd keep that effort separate.
>
> > >>
>
> > >> On Wed, Jun 9, 2021 at 9:38 AM Till Rohrmann
>
> > >> wrote:
>
> > >>
>
> > >>> Thanks for the lively discussion everyone. I have to admit that I am
>
> > not
>
> > >>> really convinced that we should call the interface Flushable and the
>
> > >>> method
>
> > >>> flush. The problem is that this method should in the first place tell
>
> > the
>
> > >>> operator that it should stop processing and flush all buffered data.
>
> > The
>
> > >>> method "flush" alone does not convey this contract very well. Maybe a
>
> > >>> more
>
> > >>> explicit name like stopProcessingAndFlush (maybe only stopProcessing)
>
> > >>> would
>
> > >>> be better. Moreover, from the OutputStream.flush method, I would
> expect
>
> > >>> that I can call this method multiple times w/o changing the state of
>
> > the
>
> > >>> stream. This is not the case here.
>
> > >>>
>
> > >>> Given that the stop processing and flush all results is such an
>
> > essential
>
> > >>> lifecycle method of an operator/UDF, I am not sure whether we should
>
> > >>> offer
>
> > >>> it as an optional interface users can implement. The problem I see is
>
> > >>> that
>
> > >>> users are not aware of it when writing their own operators/UDFs.
> Making
>
> > >>> it
>
> > >>> part of the actual interfaces makes it more explicit and easier to
>
> > >>> discover. Maybe there is a way of adding it together with a default
>
> > >>> implementation, deprecating the other methods, and then eventually
>
> > >>> removing
>
> > >>> the old methods. The last step will break APIs, though :-(
>
> > >>>
>
> > >>> Cheers,
>
> > >>> Till
>
> > >>>
>
> > >>> On Tue, Jun 8, 2021 at 6:27 PM Piotr Nowojski
>
> > >>> wrote:
>
> > >>>
>
> > >>> > Hi,
>
> > >>> >
>
> > >>> > Thanks for resuming this discussion. I think +1 for the proposal of
>
> > >>> > dropping (deprecating) `dispose()`, and adding `flush()` to the
>
> > >>> > `StreamOperator`/udfs. Semantically it would be more like new
>
> > >>> `close()` is
>
> > >>> > an equivalent of old `dispose()`. Old `close()` is an equivalent of
>
> > new
>
> > >>> > `flush() + close()`. I think it provides a relatively painless
>
> > >>> migration
>
> > >>> > path (could we write down this migration?).
>
> > >>> >
>
> > >>> > However I have some doubts about the Flushable interface. First of
>
> > >>> all,
>
> > >>> > it wouldn't work for sinks - sinks have no output. Secondly, I
> don't
>
> > >>> like
>
> > >>> > that it opens a possibility for problems like this (incompatible
>
> > output
>
> > >>> > types):
>
> > >>> > ```
>
> > >>> > public class MyMap implements MapFunction,
>
> > >>> Flushable
>
> > >>> > { ...}
>
> > >>> > ```
>
> > >>> >
>
> > >>> > Also after a quick offline discussion with Dawid, I'm not sure
>
> > anymore
>
> > >>> to
>
> > >>> > which UDFs it actually makes sense to add `flush`, as most of them
>
> > >>> > shouldn't buffer any data. Apart from Sinks, it's usually an
> operator
>
> > >>> that
>
> > >>> > is buffering the data (that holds true for AsyncFunction,
>
> > >>> ReduceFunction,
>
> > >>> > AggregateFunction, MapFunction, FilterFunction, ...). For those
>
> > >>> functions
>
> > >>> > it's difficult to buffer any data, as they have no means to control
>
> > >>> when to
>
> > >>> > emit this data. One notable exception might be (Co)ProcessFunction,
>
> > as
>
> > >>> it
>
> > >>> > can register timers on it's own. In that case I would propose to do
>
> > the
>
> > >>> > following thing:
>
> > >>> > 1. Add `flush() {}` to `Sink` function (FLIP-143 Writer interface
>
> > >>> already
>
> > >>> > has flush capabilities)
>
> > >>> > 2. Maybe add `flush(Collector)` to `(Co)ProcessFunction`, but
>
> > maybe
>
> > >>> we
>
> > >>> > can postpone it
>
> > >>> > 3. Leave other functions alone.
>
> > >>> >
>
> > >>> > After all, we could add `flush()` to other functions in the future
> if
>
> > >>> we
>
> > >>> > really find a good motivating example to do so.
>
> > >>> >
>
> > >>> > About 2. Dawid is pitching an idea to convert `ProcessFunction`
> into
>
> > a
>
> > >>> > proper `Public` API that would replace StreamOperator. We could
>
> > change
>
> > >>> > `StreamOperator` to be purely `@Internal` class/interface, and add
>
> > the
>
> > >>> > missing functionality to the `ProcessFunction` (InputSelectable,
>
> > >>> > BoundedInput, MailboxExecutor). With this, adding `flush()` to
>
> > >>> > `ProcessFunction` would make a lot of sense. But maybe that should
>
> > be a
>
> > >>> > story for another day?
>
> > >>> >
>
> > >>> > Best,
>
> > >>> > Piotrek
>
> > >>> >
>
> > >>> > pt., 4 cze 2021 o 10:36 Yun Gao  napisał(a):
>
> > >>> >
>
> > >>> >> Hi all,
>
> > >>> >>
>
> > >>> >> Very thanks @Dawid for resuming the discussion and very thanks
> @Till
>
> > >>> for
>
> > >>> >> the summary ! (and very sorry for I missed the mail and do not
>
> > >>> response
>
> > >>> >> in time...)
>
> > >>> >>
>
> > >>> >> I also agree with that we could consider the global commits latter
>
> > >>> >> separately after we have addressed the final checkpoints, and also
>
> > >>> other
>
> > >>> >> points as Till summarized.
>
> > >>> >> Currently the only case that have used the cascade commit is the
>
> > Table
>
> > >>> >> FileSystem and Hive connectors. I checked the code and found
>
> > >>> currently they
>
> > >>> >> will commit the
>
> > >>> >> last piece of data directly in endOfInput(). Although this might
>
> > emit
>
> > >>> >> repeat records if there are failover during job finishing, it
> avoids
>
> > >>> >> emitting the records in the
>
> > >>> >> notifyCheckpointComplete() after endOfInput(), thus the
> modification
>
> > >>> to
>
> > >>> >> the operator lifecycle in final checkpoints would cause
>
> > compatibility
>
> > >>> >> problem for these connectors,
>
> > >>> >> thus we do not need to modify them at the first place.
>
> > >>> >>
>
> > >>> >> 2. Regarding the operator lifecycle, I also agree with the
> proposed
>
> > >>> >> changes. To sum up, I think the operator lifecycle would become
>
> > >>> >>
>
> > >>> >> endOfInput(1)
>
> > >>> >> ...
>
> > >>> >> endOfInput(n)
>
> > >>> >> flush() --> call UDF's flush method
>
> > >>> >> if some operator requires final checkpoints
>
> > >>> >> snapshotState()
>
> > >>> >> notifyCheckpointComplete()
>
> > >>> >> end if
>
> > >>> >> close() --> call UDF's close method
>
> > >>> >>
>
> > >>> >> Since currently the close() is only called in normal finish and
>
> > >>> dispose()
>
> > >>> >> will be called in both failover and normal case, for
> compatibility,
>
> > I
>
> > >>> think
>
> > >>> >> we may
>
> > >>> >> have to postpone the change to a single close() method to version
>
> > 2.0
>
> > >>> ?
>
> > >>> >>
>
> > >>> >> 3. Regarding the name and position of flush() method, I also agree
>
> > >>> with
>
> > >>> >> that we will need a separate method to mark the termination of the
>
> > >>> whole
>
> > >>> >> stream for
>
> > >>> >> multiple-input streams. Would it be also ok if we have some
>
> > >>> modification
>
> > >>> >> to the current BoundedXXInput interfaces to
>
> > >>> >>
>
> > >>> >> interface BoundedInput {
>
> > >>> >> void endInput() // marks the end of the whole streams, as
>
> > flush()
>
> > >>> >> does.
>
> > >>> >> }
>
> > >>> >>
>
> > >>> >> @deprecated // In the future we could remove this interface
>
> > >>> >> interface BoundedOneInput extends BoundedInput {}
>
> > >>> >>
>
> > >>> >> interface BoundedMultiInput extends BoundedInput {
>
> > >>> >> void endInput(int i);
>
> > >>> >>
>
> > >>> >> default void endInput() {} // For compatibility
>
> > >>> >> }
>
> > >>> >>
>
> > >>> >> If operator/UDF does not care about the end of a single input,
> then
>
> > it
>
> > >>> >> could directly implement the BoundedInput interface. The possible
>
> > >>> >> benefit to me is that we might be able to keep only one concept
> for
>
> > >>> >> marking the end of stream, especially for the operators with only
>
> > >>> >> one input.
>
> > >>> >>
>
> > >>> >> Very thanks for all the deep insights and discussions!
>
> > >>> >>
>
> > >>> >> Best,
>
> > >>> >> Yun
>
> > >>> >>
>
> > >>> >> ------------------------------------------------------------------
>
> > >>> >> From:Dawid Wysakowicz
>
> > >>> >> Send Time:2021 Jun. 3 (Thu.) 21:21
>
> > >>> >> To:dev ; Till Rohrmann
> > >;
>
> > >>> Yun
>
> > >>> >> Gao
>
> > >>> >> Cc:Piotr Nowojski ; Guowei Ma <
>
> > >>> guowei.mgw@gmail.com>;
>
> > >>> >> Stephan Ewen
>
> > >>> >> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
>
> > >>> Finished
>
> > >>> >>
>
> > >>> >> Hi all,
>
> > >>> >>
>
> > >>> >> Thanks for the very insightful discussion. I'd like to revive the
>
> > >>> effort
>
> > >>> >> of FLIP-147. First of all, from my side I'd like to say that I am
>
> > >>> really
>
> > >>> >> interested in helping that happen in the upcoming 1.14 release.
>
> > >>> >>
>
> > >>> >> I agree with Till that the final checkpoints and global commits
> are
>
> > >>> >> mostly orthogonal. Similarly as Till, I'd suggest to first focus
> on
>
> > >>> the
>
> > >>> >> final checkpoints, while just keeping in mind we should not make
>
> > >>> >> assumptions that would make it impossible to implement the global
>
> > >>> commits.
>
> > >>> >> So far I do not see such risk from the discussion.
>
> > >>> >>
>
> > >>> >> Going back to the final checkpoints issue. I think the only
>
> > >>> outstanding
>
> > >>> >> issue is which methods we want to use for flushing/closing both
>
> > >>> operators
>
> > >>> >> and UDFs just before performing the final checkpoint. As pointed
> out
>
> > >>> to me
>
> > >>> >> by Piotr, I am mentioning UDFs here as well, because we need a way
>
> > for
>
> > >>> >> users using the Public API to benefit from the final checkpoint
>
> > (bear
>
> > >>> in
>
> > >>> >> mind that e.g. TwoPhaseCommitSinkFunction which is implemented by
>
> > our
>
> > >>> Kafka
>
> > >>> >> sink operates on the UDF level). Right now RichFunction has no
>
> > method
>
> > >>> which
>
> > >>> >> could be called just before the final checkpoint that would say
>
> > >>> "flush" all
>
> > >>> >> intermediate state now and prepare for the final checkpoint. I'd
>
> > >>> suggest
>
> > >>> >> introducing an additional interface e.g. (name to be determined)
>
> > >>> >>
>
> > >>> >> interface Flushable {
>
> > >>> >>
>
> > >>> >> void flush(Collector out)
>
> > >>> >>
>
> > >>> >> }
>
> > >>> >>
>
> > >>> >> Additionally we would need to introduce a similar method on the
>
> > >>> >> StreamOperator level. Currently we have two methods that are
> called
>
> > >>> at the
>
> > >>> >> end of operator lifecycle:
>
> > >>> >>
>
> > >>> >> -
>
> > >>> >> - close
>
> > >>> >> - dispose
>
> > >>> >>
>
> > >>> >> The usage of the two methods is a bit confusing. Dispose is
>
> > >>> responsible
>
> > >>> >> for closing all open resources and is supposed to be called in
> case
>
> > >>> of a
>
> > >>> >> failure. On the other hand the close is a combination of a
>
> > >>> non-existent
>
> > >>> >> "flush" method we need and dispose for closing resources in case
> of
>
> > a
>
> > >>> >> successful run. I'd suggest to clear it a bit. We would introduce
> a
>
> > >>> proper
>
> > >>> >> "flush" method which would be called in case of a successful
>
> > >>> finishing of
>
> > >>> >> an operator. Moreover we would make "close" deal only with closing
>
> > >>> any open
>
> > >>> >> resources, basically taking over the role of the dispose, which we
>
> > >>> would
>
> > >>> >> deprecate.
>
> > >>> >>
>
> > >>> >> Lastly, I'd like to say why I think it is better introduce a new
>
> > >>> "flush"
>
> > >>> >> method instead of using the "endInput" method of
>
> > >>> BoundedOne/MultiInput.
>
> > >>> >> That method is called per input, which means each operator would
>
> > need
>
> > >>> to
>
> > >>> >> keep track of which inputs were already closed internally and
> react
>
> > >>> >> differently if all of the inputs were closed. With an explicit
>
> > "flush"
>
> > >>> >> method we do not have such a problem as the input bookkeeping
>
> > happens
>
> > >>> on
>
> > >>> >> the StreamTask level.
>
> > >>> >>
>
> > >>> >> Let me know what you think. I'd sync with Yun Gao and if there are
>
> > no
>
> > >>> >> objections we will extend the FLIP page with necessary changes.
>
> > >>> >>
>
> > >>> >> Best,
>
> > >>> >>
>
> > >>> >> Dawid
>
> > >>> >>
>
> > >>> >>
>
> > >>> >>
>
> > >>>
>
> > >>
>
> >
>
>

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi all,

Very thanks for the warm discussions!

Regarding the change in the operator lifecycle, I also agree with adding 
the flush/drain/stopAndFlush/finish method. For the duplication between this
method and `endInput` for one input operator, with some offline disucssion with
Dawid now I also think we might instead consider dropping the BoundedOneInput
in the future and thus it should not be an issue. 

But for the compatibility problem I still have one concern: for the new lifecycle
we would remove `dispose()` and call `close()` for both normal finish and failover, if some
users use customized operator that only overrides `close()` method, with the new lifecycle
it would still compile, but might have wrong result if there are failover. Perhaps the SQL
mini-batch operator (AbstractMapBundleOperator) is an example: it finish the last mini-batch
in the close() method, if the method also get called in failover, the last mini-batch would get emitted, 
and these data would also replayed after failover, which would cause data repeatation. But considering
there might be not too much users write customized operators, would this be a blocker ? 

Regarding the name of the method, I would also tend to simpler name, perhaps finish (should be more easier
for user to understand) or drain (since the method should only be called on termination and stop-with-savepoint 
--drain, the concept would be more consistent). 

Regarding the changes in the UDF, I also agree with that to avoid the output type issue,  we may only
add flush/drain/stopAndFlush/finish method to sink function for now. In the future if there 
are more requirments we may further consider add the method to the low-level UDF like
ProcessFunction / CoProcessFunction as an advanced feature. 

Best,
Yun








 ------------------Original Mail ------------------
Sender:Arvid Heise <ar...@apache.org>
Send Date:Thu Jun 10 01:08:36 2021
Recipients:dev <de...@flink.apache.org>
CC:Yun Gao <yu...@aliyun.com>, Till Rohrmann <tr...@apache.org>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
Hi Piot,



I'm fine with just doing it on the Sink. My responses were focused on the

API (the how) not on the concept (the if). Just keep the methods on the

different places in sync, such that it is easy to introduce a common

interface later.



Re name: drain is not a reinvention as it's used quite often throughout

Flink (e.g., Mailbox, stopWithSavepoint with drain flag). flush has no link

to life-cycles: you usually do it arbitrarily often.

I like `finish` very much as it relates to JobStatus FINISHED, has a clear

tie to life-cycles, and is crisp.

I also thought about `terminate` but I'd clearly favor `finish` as the

verbs cannot be exchanged in the following: the task may terminate its

operators but the operators should finish their thing first.



On Wed, Jun 9, 2021 at 6:48 PM Piotr Nowojski  wrote:



> Hi,

>

> Arvid: What's the problem with providing `void flush()`/`void drain()` only

> in the `SinkFunction`? It would avoid the problem of typing. Why would one

> need to have it in the other `Rich***Function`s? For `flush()` to make

> sense, the entity which has this method, would need to buffer some data. I

> don't see this to be reasonable in anything but `SinkFunction`,

> `ProcessFunction` and operators.

>

> Re `flush()` vs `drain()`. Frankly, I wouldn't understand what `drain()` is

> all about without reading the java-doc, and afterwards, I would have an

> impression that someone wanted to reinvent a wheel :) `flush()` is kind of

> an industry standard for things like that. Furthermore I don't think

> `drain()` solves Till's concern (drain != stop + flush). `stopAndFlush()`

> would be better in this regard, but it also doesn't feel right. Maybe

> `finish()`?

>

> Piotrek

>

> śr., 9 cze 2021 o 11:51 Arvid Heise  napisał(a):

>

> > Hi Dawid,

> >

> > I see your point. I'd probably add drain only to Rich*Function where we

> > have the type bounds. Then we still need your Flushable interface in

> > Rich*Function<..., T> to call it efficiently but we at least avoid weird

> > type combinations. I'll have a rethink later.

> >

> > The proper solution is probably to add  to RichFunction and use Void

> > for RichSinkFunction but I'll have to understand the implications first.

> >

> > On Wed, Jun 9, 2021 at 11:37 AM Dawid Wysakowicz 
> >

> > wrote:

> >

> >> Hey,

> >>

> >> @Arvid The problem with adding the "drain/flush/stopProcessing" method

> to

> >> RichFunction is that it is not typed with the output type. At the same

> time

> >> we would most likely need a way to emit records from the method. That's

> >> originally thought about adding a typed interface which honestly I don't

> >> like that much either.

> >>

> >> On the UDF level we do not need to deprecate anything as you said. The

> >> close there already works as dispose on the Operator level. What we are

> >> suggesting is to unify that on the Operator level and deprecate the

> dispose

> >> there. @Yun I think we can already do that. We can either try to handle

> >> exceptions from the close in the case of a failure or just break it as

> it

> >> is a low-level, mostly internal API as Arvid said and also the migration

> >> would be really easy there.

> >>

> >> @Till @Arvid I am open for suggestions about the naming. I like the

> >> "drain" method.

> >>

> >> For now I'd go with @Piotr's proposal to add the "drain" method only to

> >> the SinkFunction. We think they are not immediately necessary for any of

> >> the other UDFs.

> >>

> >> Best,

> >>

> >> Dawid

> >> On 09/06/2021 11:20, Arvid Heise wrote:

> >>

> >> I have not followed the complete discussion and can't comment on the

> >> concepts. However, I have some ideas on the API changes:

> >>

> >> 1. If it's about adding additional life-cycle methods to UDFs, we should

> >> add the flush/endOfInput to RichFunction as this is the current way to

> >> define it. At this point, I don't see the need to add/change anything

> for

> >> UDFs. Since RichFunction does not have a dispose, do we even need to

> >> deprecate anything on UDF level? This would avoid having a new interface

> >> Flushable altogether (of which I'm not a big fan, see Piot's mail)

> >>

> >> 2. Further, I'd like to propose drain instead of flush as it would more

> >> align with the current nomenclature and makes the intent more obvious.

> >> However, that only works if there is no clash, so please double-check.

> >>

> >> 3. About changing methods on Operators: I'd say go ahead. It's

> >> experimental and not too hard to adjust on the user side. I also like

> the

> >> idea of beefing up ProcessFunction as a full replacement to custom

> >> operators but I'd keep that effort separate.

> >>

> >> On Wed, Jun 9, 2021 at 9:38 AM Till Rohrmann 

> >> wrote:

> >>

> >>> Thanks for the lively discussion everyone. I have to admit that I am

> not

> >>> really convinced that we should call the interface Flushable and the

> >>> method

> >>> flush. The problem is that this method should in the first place tell

> the

> >>> operator that it should stop processing and flush all buffered data.

> The

> >>> method "flush" alone does not convey this contract very well. Maybe a

> >>> more

> >>> explicit name like stopProcessingAndFlush (maybe only stopProcessing)

> >>> would

> >>> be better. Moreover, from the OutputStream.flush method, I would expect

> >>> that I can call this method multiple times w/o changing the state of

> the

> >>> stream. This is not the case here.

> >>>

> >>> Given that the stop processing and flush all results is such an

> essential

> >>> lifecycle method of an operator/UDF, I am not sure whether we should

> >>> offer

> >>> it as an optional interface users can implement. The problem I see is

> >>> that

> >>> users are not aware of it when writing their own operators/UDFs. Making

> >>> it

> >>> part of the actual interfaces makes it more explicit and easier to

> >>> discover. Maybe there is a way of adding it together with a default

> >>> implementation, deprecating the other methods, and then eventually

> >>> removing

> >>> the old methods. The last step will break APIs, though :-(

> >>>

> >>> Cheers,

> >>> Till

> >>>

> >>> On Tue, Jun 8, 2021 at 6:27 PM Piotr Nowojski 

> >>> wrote:

> >>>

> >>> > Hi,

> >>> >

> >>> > Thanks for resuming this discussion. I think +1 for the proposal of

> >>> > dropping (deprecating) `dispose()`, and adding `flush()` to the

> >>> > `StreamOperator`/udfs. Semantically it would be more like new

> >>> `close()` is

> >>> > an equivalent of old `dispose()`. Old `close()` is an equivalent of

> new

> >>> > `flush() + close()`. I think it provides a relatively painless

> >>> migration

> >>> > path (could we write down this migration?).

> >>> >

> >>> > However I have some doubts about the Flushable interface. First of

> >>> all,

> >>> > it wouldn't work for sinks - sinks have no output. Secondly, I don't

> >>> like

> >>> > that it opens a possibility for problems like this (incompatible

> output

> >>> > types):

> >>> > ```

> >>> > public class MyMap implements MapFunction,

> >>> Flushable

> >>> > { ...}

> >>> > ```

> >>> >

> >>> > Also after a quick offline discussion with Dawid, I'm not sure

> anymore

> >>> to

> >>> > which UDFs it actually makes sense to add `flush`, as most of them

> >>> > shouldn't buffer any data. Apart from Sinks, it's usually an operator

> >>> that

> >>> > is buffering the data (that holds true for AsyncFunction,

> >>> ReduceFunction,

> >>> > AggregateFunction, MapFunction, FilterFunction, ...). For those

> >>> functions

> >>> > it's difficult to buffer any data, as they have no means to control

> >>> when to

> >>> > emit this data. One notable exception might be (Co)ProcessFunction,

> as

> >>> it

> >>> > can register timers on it's own. In that case I would propose to do

> the

> >>> > following thing:

> >>> > 1. Add `flush() {}` to `Sink` function (FLIP-143 Writer interface

> >>> already

> >>> > has flush capabilities)

> >>> > 2. Maybe add `flush(Collector)` to `(Co)ProcessFunction`, but

> maybe

> >>> we

> >>> > can postpone it

> >>> > 3. Leave other functions alone.

> >>> >

> >>> > After all, we could add `flush()` to other functions in the future if

> >>> we

> >>> > really find a good motivating example to do so.

> >>> >

> >>> > About 2. Dawid is pitching an idea to convert `ProcessFunction` into

> a

> >>> > proper `Public` API that would replace StreamOperator. We could

> change

> >>> > `StreamOperator` to be purely `@Internal` class/interface, and add

> the

> >>> > missing functionality to the `ProcessFunction` (InputSelectable,

> >>> > BoundedInput, MailboxExecutor). With this, adding `flush()` to

> >>> > `ProcessFunction` would make a lot of sense. But maybe that should

> be a

> >>> > story for another day?

> >>> >

> >>> > Best,

> >>> > Piotrek

> >>> >

> >>> > pt., 4 cze 2021 o 10:36 Yun Gao  napisał(a):

> >>> >

> >>> >> Hi all,

> >>> >>

> >>> >> Very thanks @Dawid for resuming the discussion and very thanks @Till

> >>> for

> >>> >> the summary ! (and very sorry for I missed the mail and do not

> >>> response

> >>> >> in time...)

> >>> >>

> >>> >> I also agree with that we could consider the global commits latter

> >>> >> separately after we have addressed the final checkpoints, and also

> >>> other

> >>> >> points as Till summarized.

> >>> >> Currently the only case that have used the cascade commit is the

> Table

> >>> >> FileSystem and Hive connectors. I checked the code and found

> >>> currently they

> >>> >> will commit the

> >>> >> last piece of data directly in endOfInput(). Although this might

> emit

> >>> >> repeat records if there are failover during job finishing, it avoids

> >>> >> emitting the records in the

> >>> >> notifyCheckpointComplete() after endOfInput(), thus the modification

> >>> to

> >>> >> the operator lifecycle in final checkpoints would cause

> compatibility

> >>> >> problem for these connectors,

> >>> >> thus we do not need to modify them at the first place.

> >>> >>

> >>> >> 2. Regarding the operator lifecycle, I also agree with the proposed

> >>> >> changes. To sum up, I think the operator lifecycle would become

> >>> >>

> >>> >> endOfInput(1)

> >>> >> ...

> >>> >> endOfInput(n)

> >>> >> flush() --> call UDF's flush method

> >>> >> if some operator requires final checkpoints

> >>> >> snapshotState()

> >>> >> notifyCheckpointComplete()

> >>> >> end if

> >>> >> close() --> call UDF's close method

> >>> >>

> >>> >> Since currently the close() is only called in normal finish and

> >>> dispose()

> >>> >> will be called in both failover and normal case, for compatibility,

> I

> >>> think

> >>> >> we may

> >>> >> have to postpone the change to a single close() method to version

> 2.0

> >>> ?

> >>> >>

> >>> >> 3. Regarding the name and position of flush() method, I also agree

> >>> with

> >>> >> that we will need a separate method to mark the termination of the

> >>> whole

> >>> >> stream for

> >>> >> multiple-input streams. Would it be also ok if we have some

> >>> modification

> >>> >> to the current BoundedXXInput interfaces to

> >>> >>

> >>> >> interface BoundedInput {

> >>> >> void endInput() // marks the end of the whole streams, as

> flush()

> >>> >> does.

> >>> >> }

> >>> >>

> >>> >> @deprecated // In the future we could remove this interface

> >>> >> interface BoundedOneInput extends BoundedInput {}

> >>> >>

> >>> >> interface BoundedMultiInput extends BoundedInput {

> >>> >> void endInput(int i);

> >>> >>

> >>> >> default void endInput() {} // For compatibility

> >>> >> }

> >>> >>

> >>> >> If operator/UDF does not care about the end of a single input, then

> it

> >>> >> could directly implement the BoundedInput interface. The possible

> >>> >> benefit to me is that we might be able to keep only one concept for

> >>> >> marking the end of stream, especially for the operators with only

> >>> >> one input.

> >>> >>

> >>> >> Very thanks for all the deep insights and discussions!

> >>> >>

> >>> >> Best,

> >>> >> Yun

> >>> >>

> >>> >> ------------------------------------------------------------------

> >>> >> From:Dawid Wysakowicz 

> >>> >> Send Time:2021 Jun. 3 (Thu.) 21:21

> >>> >> To:dev ; Till Rohrmann 
> >;

> >>> Yun

> >>> >> Gao 

> >>> >> Cc:Piotr Nowojski ; Guowei Ma <

> >>> guowei.mgw@gmail.com>;

> >>> >> Stephan Ewen 

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

> >>> Finished

> >>> >>

> >>> >> Hi all,

> >>> >>

> >>> >> Thanks for the very insightful discussion. I'd like to revive the

> >>> effort

> >>> >> of FLIP-147. First of all, from my side I'd like to say that I am

> >>> really

> >>> >> interested in helping that happen in the upcoming 1.14 release.

> >>> >>

> >>> >> I agree with Till that the final checkpoints and global commits are

> >>> >> mostly orthogonal. Similarly as Till, I'd suggest to first focus on

> >>> the

> >>> >> final checkpoints, while just keeping in mind we should not make

> >>> >> assumptions that would make it impossible to implement the global

> >>> commits.

> >>> >> So far I do not see such risk from the discussion.

> >>> >>

> >>> >> Going back to the final checkpoints issue. I think the only

> >>> outstanding

> >>> >> issue is which methods we want to use for flushing/closing both

> >>> operators

> >>> >> and UDFs just before performing the final checkpoint. As pointed out

> >>> to me

> >>> >> by Piotr, I am mentioning UDFs here as well, because we need a way

> for

> >>> >> users using the Public API to benefit from the final checkpoint

> (bear

> >>> in

> >>> >> mind that e.g. TwoPhaseCommitSinkFunction which is implemented by

> our

> >>> Kafka

> >>> >> sink operates on the UDF level). Right now RichFunction has no

> method

> >>> which

> >>> >> could be called just before the final checkpoint that would say

> >>> "flush" all

> >>> >> intermediate state now and prepare for the final checkpoint. I'd

> >>> suggest

> >>> >> introducing an additional interface e.g. (name to be determined)

> >>> >>

> >>> >> interface Flushable {

> >>> >>

> >>> >> void flush(Collector out)

> >>> >>

> >>> >> }

> >>> >>

> >>> >> Additionally we would need to introduce a similar method on the

> >>> >> StreamOperator level. Currently we have two methods that are called

> >>> at the

> >>> >> end of operator lifecycle:

> >>> >>

> >>> >> -

> >>> >> - close

> >>> >> - dispose

> >>> >>

> >>> >> The usage of the two methods is a bit confusing. Dispose is

> >>> responsible

> >>> >> for closing all open resources and is supposed to be called in case

> >>> of a

> >>> >> failure. On the other hand the close is a combination of a

> >>> non-existent

> >>> >> "flush" method we need and dispose for closing resources in case of

> a

> >>> >> successful run. I'd suggest to clear it a bit. We would introduce a

> >>> proper

> >>> >> "flush" method which would be called in case of a successful

> >>> finishing of

> >>> >> an operator. Moreover we would make "close" deal only with closing

> >>> any open

> >>> >> resources, basically taking over the role of the dispose, which we

> >>> would

> >>> >> deprecate.

> >>> >>

> >>> >> Lastly, I'd like to say why I think it is better introduce a new

> >>> "flush"

> >>> >> method instead of using the "endInput" method of

> >>> BoundedOne/MultiInput.

> >>> >> That method is called per input, which means each operator would

> need

> >>> to

> >>> >> keep track of which inputs were already closed internally and react

> >>> >> differently if all of the inputs were closed. With an explicit

> "flush"

> >>> >> method we do not have such a problem as the input bookkeeping

> happens

> >>> on

> >>> >> the StreamTask level.

> >>> >>

> >>> >> Let me know what you think. I'd sync with Yun Gao and if there are

> no

> >>> >> objections we will extend the FLIP page with necessary changes.

> >>> >>

> >>> >> Best,

> >>> >>

> >>> >> Dawid

> >>> >>

> >>> >>

> >>> >>

> >>>

> >>

>


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

Posted by Arvid Heise <ar...@apache.org>.
Hi Piot,

I'm fine with just doing it on the Sink. My responses were focused on the
API (the how) not on the concept (the if). Just keep the methods on the
different places in sync, such that it is easy to introduce a common
interface later.

Re name: drain is not a reinvention as it's used quite often throughout
Flink (e.g., Mailbox, stopWithSavepoint with drain flag). flush has no link
to life-cycles: you usually do it arbitrarily often.
I like `finish` very much as it relates to JobStatus FINISHED, has a clear
tie to life-cycles, and is crisp.
I also thought about `terminate` but I'd clearly favor `finish` as the
verbs cannot be exchanged in the following: the task may terminate its
operators but the operators should finish their thing first.

On Wed, Jun 9, 2021 at 6:48 PM Piotr Nowojski <pn...@apache.org> wrote:

> Hi,
>
> Arvid: What's the problem with providing `void flush()`/`void drain()` only
> in the `SinkFunction`? It would avoid the problem of typing. Why would one
> need to have it in the other `Rich***Function`s? For `flush()` to make
> sense, the entity which has this method, would need to buffer some data. I
> don't see this to be reasonable in anything but `SinkFunction`,
> `ProcessFunction` and operators.
>
> Re `flush()` vs `drain()`. Frankly, I wouldn't understand what `drain()` is
> all about without reading the java-doc, and afterwards, I would have an
> impression that someone wanted to reinvent a wheel :) `flush()` is kind of
> an industry standard for things like that. Furthermore I don't think
> `drain()` solves Till's concern (drain != stop + flush). `stopAndFlush()`
> would be better in this regard, but it also doesn't feel right. Maybe
> `finish()`?
>
> Piotrek
>
> śr., 9 cze 2021 o 11:51 Arvid Heise <ar...@apache.org> napisał(a):
>
> > Hi Dawid,
> >
> > I see your point. I'd probably add drain only to Rich*Function where we
> > have the type bounds. Then we still need your Flushable<T> interface in
> > Rich*Function<..., T> to call it efficiently but we at least avoid weird
> > type combinations. I'll have a rethink later.
> >
> > The proper solution is probably to add <OUT> to RichFunction and use Void
> > for RichSinkFunction but I'll have to understand the implications first.
> >
> > On Wed, Jun 9, 2021 at 11:37 AM Dawid Wysakowicz <dwysakowicz@apache.org
> >
> > wrote:
> >
> >> Hey,
> >>
> >> @Arvid The problem with adding the "drain/flush/stopProcessing" method
> to
> >> RichFunction is that it is not typed with the output type. At the same
> time
> >> we would most likely need a way to emit records from the method. That's
> >> originally thought about adding a typed interface which honestly I don't
> >> like that much either.
> >>
> >> On the UDF level we do not need to deprecate anything as you said. The
> >> close there already works as dispose on the Operator level. What we are
> >> suggesting is to unify that on the Operator level and deprecate the
> dispose
> >> there. @Yun I think we can already do that. We can either try to handle
> >> exceptions from the close in the case of a failure or just break it as
> it
> >> is a low-level, mostly internal API as Arvid said and also the migration
> >> would be really easy there.
> >>
> >> @Till @Arvid I am open for suggestions about the naming. I like the
> >> "drain" method.
> >>
> >> For now I'd go with @Piotr's proposal to add the "drain" method only to
> >> the SinkFunction. We think they are not immediately necessary for any of
> >> the other UDFs.
> >>
> >> Best,
> >>
> >> Dawid
> >> On 09/06/2021 11:20, Arvid Heise wrote:
> >>
> >> I have not followed the complete discussion and can't comment on the
> >> concepts. However, I have some ideas on the API changes:
> >>
> >> 1. If it's about adding additional life-cycle methods to UDFs, we should
> >> add the flush/endOfInput to RichFunction as this is the current way to
> >> define it. At this point, I don't see the need to add/change anything
> for
> >> UDFs. Since RichFunction does not have a dispose, do we even need to
> >> deprecate anything on UDF level? This would avoid having a new interface
> >> Flushable altogether (of which I'm not a big fan, see Piot's mail)
> >>
> >> 2. Further, I'd like to propose drain instead of flush as it would more
> >> align with the current nomenclature and makes the intent more obvious.
> >> However, that only works if there is no clash, so please double-check.
> >>
> >> 3. About changing methods on Operators: I'd say go ahead. It's
> >> experimental and not too hard to adjust on the user side. I also like
> the
> >> idea of beefing up ProcessFunction as a full replacement to custom
> >> operators but I'd keep that effort separate.
> >>
> >> On Wed, Jun 9, 2021 at 9:38 AM Till Rohrmann <tr...@apache.org>
> >> wrote:
> >>
> >>> Thanks for the lively discussion everyone. I have to admit that I am
> not
> >>> really convinced that we should call the interface Flushable and the
> >>> method
> >>> flush. The problem is that this method should in the first place tell
> the
> >>> operator that it should stop processing and flush all buffered data.
> The
> >>> method "flush" alone does not convey this contract very well. Maybe a
> >>> more
> >>> explicit name like stopProcessingAndFlush (maybe only stopProcessing)
> >>> would
> >>> be better. Moreover, from the OutputStream.flush method, I would expect
> >>> that I can call this method multiple times w/o changing the state of
> the
> >>> stream. This is not the case here.
> >>>
> >>> Given that the stop processing and flush all results is such an
> essential
> >>> lifecycle method of an operator/UDF, I am not sure whether we should
> >>> offer
> >>> it as an optional interface users can implement. The problem I see is
> >>> that
> >>> users are not aware of it when writing their own operators/UDFs. Making
> >>> it
> >>> part of the actual interfaces makes it more explicit and easier to
> >>> discover. Maybe there is a way of adding it together with a default
> >>> implementation, deprecating the other methods, and then eventually
> >>> removing
> >>> the old methods. The last step will break APIs, though :-(
> >>>
> >>> Cheers,
> >>> Till
> >>>
> >>> On Tue, Jun 8, 2021 at 6:27 PM Piotr Nowojski <pn...@apache.org>
> >>> wrote:
> >>>
> >>> > Hi,
> >>> >
> >>> > Thanks for resuming this discussion. I think +1 for the proposal of
> >>> > dropping (deprecating) `dispose()`, and adding `flush()` to the
> >>> > `StreamOperator`/udfs. Semantically it would be more like new
> >>> `close()` is
> >>> > an equivalent of old `dispose()`. Old `close()` is an equivalent of
> new
> >>> > `flush() + close()`. I think it provides a relatively painless
> >>> migration
> >>> > path (could we write down this migration?).
> >>> >
> >>> > However I have some doubts about the Flushable<T> interface. First of
> >>> all,
> >>> > it wouldn't work for sinks - sinks have no output. Secondly, I don't
> >>> like
> >>> > that it opens a possibility for problems like this (incompatible
> output
> >>> > types):
> >>> > ```
> >>> > public class MyMap implements MapFunction<String, Long>,
> >>> Flushable<Double>
> >>> > { ...}
> >>> > ```
> >>> >
> >>> > Also after a quick offline discussion with Dawid, I'm not sure
> anymore
> >>> to
> >>> > which UDFs it actually makes sense to add `flush`, as most of them
> >>> > shouldn't buffer any data. Apart from Sinks, it's usually an operator
> >>> that
> >>> > is buffering the data (that holds true for AsyncFunction,
> >>> ReduceFunction,
> >>> > AggregateFunction, MapFunction, FilterFunction, ...). For those
> >>> functions
> >>> > it's difficult to buffer any data, as they have no means to control
> >>> when to
> >>> > emit this data. One notable exception might be (Co)ProcessFunction,
> as
> >>> it
> >>> > can register timers on it's own. In that case I would propose to do
> the
> >>> > following thing:
> >>> > 1. Add `flush() {}` to `Sink` function (FLIP-143 Writer interface
> >>> already
> >>> > has flush capabilities)
> >>> > 2. Maybe add `flush(Collector<O>)` to `(Co)ProcessFunction`, but
> maybe
> >>> we
> >>> > can postpone it
> >>> > 3. Leave other functions alone.
> >>> >
> >>> > After all, we could add `flush()` to other functions in the future if
> >>> we
> >>> > really find a good motivating example to do so.
> >>> >
> >>> > About 2. Dawid is pitching an idea to convert `ProcessFunction` into
> a
> >>> > proper `Public` API that would replace StreamOperator. We could
> change
> >>> > `StreamOperator` to be purely `@Internal` class/interface, and add
> the
> >>> > missing functionality to the `ProcessFunction` (InputSelectable,
> >>> > BoundedInput, MailboxExecutor). With this, adding `flush()` to
> >>> > `ProcessFunction` would make a lot of sense. But maybe that should
> be a
> >>> > story for another day?
> >>> >
> >>> > Best,
> >>> > Piotrek
> >>> >
> >>> > pt., 4 cze 2021 o 10:36 Yun Gao <yu...@aliyun.com> napisał(a):
> >>> >
> >>> >> Hi all,
> >>> >>
> >>> >> Very thanks @Dawid for resuming the discussion and very thanks @Till
> >>> for
> >>> >> the summary ! (and very sorry for I missed the mail and do not
> >>> response
> >>> >> in time...)
> >>> >>
> >>> >> I also agree with that we could consider the global commits latter
> >>> >> separately after we have addressed the final checkpoints, and also
> >>> other
> >>> >> points as Till summarized.
> >>> >> Currently the only case that have used the cascade commit is the
> Table
> >>> >> FileSystem and Hive connectors. I checked the code and found
> >>> currently they
> >>> >> will commit the
> >>> >> last piece of data directly  in endOfInput(). Although this might
> emit
> >>> >> repeat records if there are failover during job finishing, it avoids
> >>> >> emitting the records in the
> >>> >> notifyCheckpointComplete() after endOfInput(), thus the modification
> >>> to
> >>> >> the operator lifecycle in final checkpoints would cause
> compatibility
> >>> >> problem for these connectors,
> >>> >> thus we do not need to modify them at the first place.
> >>> >>
> >>> >> 2. Regarding the operator lifecycle, I also agree with the proposed
> >>> >> changes. To sum up, I think the operator lifecycle would become
> >>> >>
> >>> >> endOfInput(1)
> >>> >> ...
> >>> >> endOfInput(n)
> >>> >> flush() --> call UDF's flush method
> >>> >> if some operator requires final checkpoints
> >>> >>     snapshotState()
> >>> >>     notifyCheckpointComplete()
> >>> >> end if
> >>> >> close() --> call UDF's close method
> >>> >>
> >>> >> Since currently the close() is only called in normal finish and
> >>> dispose()
> >>> >> will be called in both failover and normal case, for compatibility,
> I
> >>> think
> >>> >> we may
> >>> >> have to postpone the change to a single close() method to version
> 2.0
> >>> ?
> >>> >>
> >>> >> 3. Regarding the name and position of flush() method, I also agree
> >>> with
> >>> >> that we will need a separate method to mark the termination of the
> >>> whole
> >>> >> stream for
> >>> >> multiple-input streams. Would it be also ok if we have some
> >>> modification
> >>> >> to the current BoundedXXInput interfaces to
> >>> >>
> >>> >> interface BoundedInput {
> >>> >>     void endInput() // marks the end of the whole streams, as
> flush()
> >>> >> does.
> >>> >> }
> >>> >>
> >>> >> @deprecated // In the future we could remove this interface
> >>> >> interface BoundedOneInput extends BoundedInput {}
> >>> >>
> >>> >> interface BoundedMultiInput extends BoundedInput {
> >>> >>       void endInput(int i);
> >>> >>
> >>> >>       default void endInput() {} // For compatibility
> >>> >> }
> >>> >>
> >>> >> If operator/UDF does not care about the end of a single input, then
> it
> >>> >> could directly implement the BoundedInput interface. The possible
> >>> >> benefit to me is that we might be able to keep only one concept for
> >>> >> marking the end of stream, especially for the operators with only
> >>> >> one input.
> >>> >>
> >>> >> Very thanks for all the deep insights and discussions!
> >>> >>
> >>> >> Best,
> >>> >> Yun
> >>> >>
> >>> >> ------------------------------------------------------------------
> >>> >> From:Dawid Wysakowicz <dw...@apache.org>
> >>> >> Send Time:2021 Jun. 3 (Thu.) 21:21
> >>> >> To:dev <de...@flink.apache.org>; Till Rohrmann <trohrmann@apache.org
> >;
> >>> Yun
> >>> >> Gao <yu...@aliyun.com>
> >>> >> Cc:Piotr Nowojski <pn...@apache.org>; Guowei Ma <
> >>> guowei.mgw@gmail.com>;
> >>> >> Stephan Ewen <se...@apache.org>
> >>> >> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> >>> Finished
> >>> >>
> >>> >> Hi all,
> >>> >>
> >>> >> Thanks for the very insightful discussion. I'd like to revive the
> >>> effort
> >>> >> of FLIP-147. First of all, from my side I'd like to say that I am
> >>> really
> >>> >> interested in helping that happen in the upcoming 1.14 release.
> >>> >>
> >>> >> I agree with Till that the final checkpoints and global commits are
> >>> >> mostly orthogonal. Similarly as Till, I'd suggest to first focus on
> >>> the
> >>> >> final checkpoints, while just keeping in mind we should not make
> >>> >> assumptions that would make it impossible to implement the global
> >>> commits.
> >>> >> So far I do not see such risk from the discussion.
> >>> >>
> >>> >> Going back to the final checkpoints issue. I think the only
> >>> outstanding
> >>> >> issue is which methods we want to use for flushing/closing both
> >>> operators
> >>> >> and UDFs just before performing the final checkpoint. As pointed out
> >>> to me
> >>> >> by Piotr, I am mentioning UDFs here as well, because we need a way
> for
> >>> >> users using the Public API to benefit from the final checkpoint
> (bear
> >>> in
> >>> >> mind that e.g. TwoPhaseCommitSinkFunction which is implemented by
> our
> >>> Kafka
> >>> >> sink operates on the UDF level). Right now RichFunction has no
> method
> >>> which
> >>> >> could be called just before the final checkpoint that would say
> >>> "flush" all
> >>> >> intermediate state now and prepare for the final checkpoint. I'd
> >>> suggest
> >>> >> introducing an additional interface e.g. (name to be determined)
> >>> >>
> >>> >> interface Flushable<T> {
> >>> >>
> >>> >>    void flush(Collector<T> out)
> >>> >>
> >>> >> }
> >>> >>
> >>> >> Additionally we would need to introduce a similar method on the
> >>> >> StreamOperator level. Currently we have two methods that are called
> >>> at the
> >>> >> end of operator lifecycle:
> >>> >>
> >>> >>    -
> >>> >>    - close
> >>> >>    - dispose
> >>> >>
> >>> >> The usage of the two methods is a bit confusing. Dispose is
> >>> responsible
> >>> >> for closing all open resources and is supposed to be called in case
> >>> of a
> >>> >> failure. On the other hand the close is a combination of a
> >>> non-existent
> >>> >> "flush" method we need and dispose for closing resources in case of
> a
> >>> >> successful run. I'd suggest to clear it a bit. We would introduce a
> >>> proper
> >>> >> "flush" method which would be called in case of a successful
> >>> finishing of
> >>> >> an operator. Moreover we would make "close" deal only with closing
> >>> any open
> >>> >> resources, basically taking over the role of the dispose, which we
> >>> would
> >>> >> deprecate.
> >>> >>
> >>> >> Lastly, I'd like to say why I think it is better introduce a new
> >>> "flush"
> >>> >> method instead of using the "endInput" method of
> >>> BoundedOne/MultiInput.
> >>> >> That method is called per input, which means each operator would
> need
> >>> to
> >>> >> keep track of which inputs were already closed internally and react
> >>> >> differently if all of the inputs were closed. With an explicit
> "flush"
> >>> >> method we do not have such a problem as the input bookkeeping
> happens
> >>> on
> >>> >> the StreamTask level.
> >>> >>
> >>> >> Let me know what you think. I'd sync with Yun Gao and if there are
> no
> >>> >> objections we will extend the FLIP page with necessary changes.
> >>> >>
> >>> >> Best,
> >>> >>
> >>> >> Dawid
> >>> >>
> >>> >>
> >>> >>
> >>>
> >>
>

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

Posted by Piotr Nowojski <pn...@apache.org>.
Hi,

Arvid: What's the problem with providing `void flush()`/`void drain()` only
in the `SinkFunction`? It would avoid the problem of typing. Why would one
need to have it in the other `Rich***Function`s? For `flush()` to make
sense, the entity which has this method, would need to buffer some data. I
don't see this to be reasonable in anything but `SinkFunction`,
`ProcessFunction` and operators.

Re `flush()` vs `drain()`. Frankly, I wouldn't understand what `drain()` is
all about without reading the java-doc, and afterwards, I would have an
impression that someone wanted to reinvent a wheel :) `flush()` is kind of
an industry standard for things like that. Furthermore I don't think
`drain()` solves Till's concern (drain != stop + flush). `stopAndFlush()`
would be better in this regard, but it also doesn't feel right. Maybe
`finish()`?

Piotrek

śr., 9 cze 2021 o 11:51 Arvid Heise <ar...@apache.org> napisał(a):

> Hi Dawid,
>
> I see your point. I'd probably add drain only to Rich*Function where we
> have the type bounds. Then we still need your Flushable<T> interface in
> Rich*Function<..., T> to call it efficiently but we at least avoid weird
> type combinations. I'll have a rethink later.
>
> The proper solution is probably to add <OUT> to RichFunction and use Void
> for RichSinkFunction but I'll have to understand the implications first.
>
> On Wed, Jun 9, 2021 at 11:37 AM Dawid Wysakowicz <dw...@apache.org>
> wrote:
>
>> Hey,
>>
>> @Arvid The problem with adding the "drain/flush/stopProcessing" method to
>> RichFunction is that it is not typed with the output type. At the same time
>> we would most likely need a way to emit records from the method. That's
>> originally thought about adding a typed interface which honestly I don't
>> like that much either.
>>
>> On the UDF level we do not need to deprecate anything as you said. The
>> close there already works as dispose on the Operator level. What we are
>> suggesting is to unify that on the Operator level and deprecate the dispose
>> there. @Yun I think we can already do that. We can either try to handle
>> exceptions from the close in the case of a failure or just break it as it
>> is a low-level, mostly internal API as Arvid said and also the migration
>> would be really easy there.
>>
>> @Till @Arvid I am open for suggestions about the naming. I like the
>> "drain" method.
>>
>> For now I'd go with @Piotr's proposal to add the "drain" method only to
>> the SinkFunction. We think they are not immediately necessary for any of
>> the other UDFs.
>>
>> Best,
>>
>> Dawid
>> On 09/06/2021 11:20, Arvid Heise wrote:
>>
>> I have not followed the complete discussion and can't comment on the
>> concepts. However, I have some ideas on the API changes:
>>
>> 1. If it's about adding additional life-cycle methods to UDFs, we should
>> add the flush/endOfInput to RichFunction as this is the current way to
>> define it. At this point, I don't see the need to add/change anything for
>> UDFs. Since RichFunction does not have a dispose, do we even need to
>> deprecate anything on UDF level? This would avoid having a new interface
>> Flushable altogether (of which I'm not a big fan, see Piot's mail)
>>
>> 2. Further, I'd like to propose drain instead of flush as it would more
>> align with the current nomenclature and makes the intent more obvious.
>> However, that only works if there is no clash, so please double-check.
>>
>> 3. About changing methods on Operators: I'd say go ahead. It's
>> experimental and not too hard to adjust on the user side. I also like the
>> idea of beefing up ProcessFunction as a full replacement to custom
>> operators but I'd keep that effort separate.
>>
>> On Wed, Jun 9, 2021 at 9:38 AM Till Rohrmann <tr...@apache.org>
>> wrote:
>>
>>> Thanks for the lively discussion everyone. I have to admit that I am not
>>> really convinced that we should call the interface Flushable and the
>>> method
>>> flush. The problem is that this method should in the first place tell the
>>> operator that it should stop processing and flush all buffered data. The
>>> method "flush" alone does not convey this contract very well. Maybe a
>>> more
>>> explicit name like stopProcessingAndFlush (maybe only stopProcessing)
>>> would
>>> be better. Moreover, from the OutputStream.flush method, I would expect
>>> that I can call this method multiple times w/o changing the state of the
>>> stream. This is not the case here.
>>>
>>> Given that the stop processing and flush all results is such an essential
>>> lifecycle method of an operator/UDF, I am not sure whether we should
>>> offer
>>> it as an optional interface users can implement. The problem I see is
>>> that
>>> users are not aware of it when writing their own operators/UDFs. Making
>>> it
>>> part of the actual interfaces makes it more explicit and easier to
>>> discover. Maybe there is a way of adding it together with a default
>>> implementation, deprecating the other methods, and then eventually
>>> removing
>>> the old methods. The last step will break APIs, though :-(
>>>
>>> Cheers,
>>> Till
>>>
>>> On Tue, Jun 8, 2021 at 6:27 PM Piotr Nowojski <pn...@apache.org>
>>> wrote:
>>>
>>> > Hi,
>>> >
>>> > Thanks for resuming this discussion. I think +1 for the proposal of
>>> > dropping (deprecating) `dispose()`, and adding `flush()` to the
>>> > `StreamOperator`/udfs. Semantically it would be more like new
>>> `close()` is
>>> > an equivalent of old `dispose()`. Old `close()` is an equivalent of new
>>> > `flush() + close()`. I think it provides a relatively painless
>>> migration
>>> > path (could we write down this migration?).
>>> >
>>> > However I have some doubts about the Flushable<T> interface. First of
>>> all,
>>> > it wouldn't work for sinks - sinks have no output. Secondly, I don't
>>> like
>>> > that it opens a possibility for problems like this (incompatible output
>>> > types):
>>> > ```
>>> > public class MyMap implements MapFunction<String, Long>,
>>> Flushable<Double>
>>> > { ...}
>>> > ```
>>> >
>>> > Also after a quick offline discussion with Dawid, I'm not sure anymore
>>> to
>>> > which UDFs it actually makes sense to add `flush`, as most of them
>>> > shouldn't buffer any data. Apart from Sinks, it's usually an operator
>>> that
>>> > is buffering the data (that holds true for AsyncFunction,
>>> ReduceFunction,
>>> > AggregateFunction, MapFunction, FilterFunction, ...). For those
>>> functions
>>> > it's difficult to buffer any data, as they have no means to control
>>> when to
>>> > emit this data. One notable exception might be (Co)ProcessFunction, as
>>> it
>>> > can register timers on it's own. In that case I would propose to do the
>>> > following thing:
>>> > 1. Add `flush() {}` to `Sink` function (FLIP-143 Writer interface
>>> already
>>> > has flush capabilities)
>>> > 2. Maybe add `flush(Collector<O>)` to `(Co)ProcessFunction`, but maybe
>>> we
>>> > can postpone it
>>> > 3. Leave other functions alone.
>>> >
>>> > After all, we could add `flush()` to other functions in the future if
>>> we
>>> > really find a good motivating example to do so.
>>> >
>>> > About 2. Dawid is pitching an idea to convert `ProcessFunction` into a
>>> > proper `Public` API that would replace StreamOperator. We could change
>>> > `StreamOperator` to be purely `@Internal` class/interface, and add the
>>> > missing functionality to the `ProcessFunction` (InputSelectable,
>>> > BoundedInput, MailboxExecutor). With this, adding `flush()` to
>>> > `ProcessFunction` would make a lot of sense. But maybe that should be a
>>> > story for another day?
>>> >
>>> > Best,
>>> > Piotrek
>>> >
>>> > pt., 4 cze 2021 o 10:36 Yun Gao <yu...@aliyun.com> napisał(a):
>>> >
>>> >> Hi all,
>>> >>
>>> >> Very thanks @Dawid for resuming the discussion and very thanks @Till
>>> for
>>> >> the summary ! (and very sorry for I missed the mail and do not
>>> response
>>> >> in time...)
>>> >>
>>> >> I also agree with that we could consider the global commits latter
>>> >> separately after we have addressed the final checkpoints, and also
>>> other
>>> >> points as Till summarized.
>>> >> Currently the only case that have used the cascade commit is the Table
>>> >> FileSystem and Hive connectors. I checked the code and found
>>> currently they
>>> >> will commit the
>>> >> last piece of data directly  in endOfInput(). Although this might emit
>>> >> repeat records if there are failover during job finishing, it avoids
>>> >> emitting the records in the
>>> >> notifyCheckpointComplete() after endOfInput(), thus the modification
>>> to
>>> >> the operator lifecycle in final checkpoints would cause compatibility
>>> >> problem for these connectors,
>>> >> thus we do not need to modify them at the first place.
>>> >>
>>> >> 2. Regarding the operator lifecycle, I also agree with the proposed
>>> >> changes. To sum up, I think the operator lifecycle would become
>>> >>
>>> >> endOfInput(1)
>>> >> ...
>>> >> endOfInput(n)
>>> >> flush() --> call UDF's flush method
>>> >> if some operator requires final checkpoints
>>> >>     snapshotState()
>>> >>     notifyCheckpointComplete()
>>> >> end if
>>> >> close() --> call UDF's close method
>>> >>
>>> >> Since currently the close() is only called in normal finish and
>>> dispose()
>>> >> will be called in both failover and normal case, for compatibility, I
>>> think
>>> >> we may
>>> >> have to postpone the change to a single close() method to version 2.0
>>> ?
>>> >>
>>> >> 3. Regarding the name and position of flush() method, I also agree
>>> with
>>> >> that we will need a separate method to mark the termination of the
>>> whole
>>> >> stream for
>>> >> multiple-input streams. Would it be also ok if we have some
>>> modification
>>> >> to the current BoundedXXInput interfaces to
>>> >>
>>> >> interface BoundedInput {
>>> >>     void endInput() // marks the end of the whole streams, as flush()
>>> >> does.
>>> >> }
>>> >>
>>> >> @deprecated // In the future we could remove this interface
>>> >> interface BoundedOneInput extends BoundedInput {}
>>> >>
>>> >> interface BoundedMultiInput extends BoundedInput {
>>> >>       void endInput(int i);
>>> >>
>>> >>       default void endInput() {} // For compatibility
>>> >> }
>>> >>
>>> >> If operator/UDF does not care about the end of a single input, then it
>>> >> could directly implement the BoundedInput interface. The possible
>>> >> benefit to me is that we might be able to keep only one concept for
>>> >> marking the end of stream, especially for the operators with only
>>> >> one input.
>>> >>
>>> >> Very thanks for all the deep insights and discussions!
>>> >>
>>> >> Best,
>>> >> Yun
>>> >>
>>> >> ------------------------------------------------------------------
>>> >> From:Dawid Wysakowicz <dw...@apache.org>
>>> >> Send Time:2021 Jun. 3 (Thu.) 21:21
>>> >> To:dev <de...@flink.apache.org>; Till Rohrmann <tr...@apache.org>;
>>> Yun
>>> >> Gao <yu...@aliyun.com>
>>> >> Cc:Piotr Nowojski <pn...@apache.org>; Guowei Ma <
>>> guowei.mgw@gmail.com>;
>>> >> Stephan Ewen <se...@apache.org>
>>> >> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
>>> Finished
>>> >>
>>> >> Hi all,
>>> >>
>>> >> Thanks for the very insightful discussion. I'd like to revive the
>>> effort
>>> >> of FLIP-147. First of all, from my side I'd like to say that I am
>>> really
>>> >> interested in helping that happen in the upcoming 1.14 release.
>>> >>
>>> >> I agree with Till that the final checkpoints and global commits are
>>> >> mostly orthogonal. Similarly as Till, I'd suggest to first focus on
>>> the
>>> >> final checkpoints, while just keeping in mind we should not make
>>> >> assumptions that would make it impossible to implement the global
>>> commits.
>>> >> So far I do not see such risk from the discussion.
>>> >>
>>> >> Going back to the final checkpoints issue. I think the only
>>> outstanding
>>> >> issue is which methods we want to use for flushing/closing both
>>> operators
>>> >> and UDFs just before performing the final checkpoint. As pointed out
>>> to me
>>> >> by Piotr, I am mentioning UDFs here as well, because we need a way for
>>> >> users using the Public API to benefit from the final checkpoint (bear
>>> in
>>> >> mind that e.g. TwoPhaseCommitSinkFunction which is implemented by our
>>> Kafka
>>> >> sink operates on the UDF level). Right now RichFunction has no method
>>> which
>>> >> could be called just before the final checkpoint that would say
>>> "flush" all
>>> >> intermediate state now and prepare for the final checkpoint. I'd
>>> suggest
>>> >> introducing an additional interface e.g. (name to be determined)
>>> >>
>>> >> interface Flushable<T> {
>>> >>
>>> >>    void flush(Collector<T> out)
>>> >>
>>> >> }
>>> >>
>>> >> Additionally we would need to introduce a similar method on the
>>> >> StreamOperator level. Currently we have two methods that are called
>>> at the
>>> >> end of operator lifecycle:
>>> >>
>>> >>    -
>>> >>    - close
>>> >>    - dispose
>>> >>
>>> >> The usage of the two methods is a bit confusing. Dispose is
>>> responsible
>>> >> for closing all open resources and is supposed to be called in case
>>> of a
>>> >> failure. On the other hand the close is a combination of a
>>> non-existent
>>> >> "flush" method we need and dispose for closing resources in case of a
>>> >> successful run. I'd suggest to clear it a bit. We would introduce a
>>> proper
>>> >> "flush" method which would be called in case of a successful
>>> finishing of
>>> >> an operator. Moreover we would make "close" deal only with closing
>>> any open
>>> >> resources, basically taking over the role of the dispose, which we
>>> would
>>> >> deprecate.
>>> >>
>>> >> Lastly, I'd like to say why I think it is better introduce a new
>>> "flush"
>>> >> method instead of using the "endInput" method of
>>> BoundedOne/MultiInput.
>>> >> That method is called per input, which means each operator would need
>>> to
>>> >> keep track of which inputs were already closed internally and react
>>> >> differently if all of the inputs were closed. With an explicit "flush"
>>> >> method we do not have such a problem as the input bookkeeping happens
>>> on
>>> >> the StreamTask level.
>>> >>
>>> >> Let me know what you think. I'd sync with Yun Gao and if there are no
>>> >> objections we will extend the FLIP page with necessary changes.
>>> >>
>>> >> Best,
>>> >>
>>> >> Dawid
>>> >>
>>> >>
>>> >>
>>>
>>

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

Posted by Arvid Heise <ar...@apache.org>.
Hi Dawid,

I see your point. I'd probably add drain only to Rich*Function where we
have the type bounds. Then we still need your Flushable<T> interface in
Rich*Function<..., T> to call it efficiently but we at least avoid weird
type combinations. I'll have a rethink later.

The proper solution is probably to add <OUT> to RichFunction and use Void
for RichSinkFunction but I'll have to understand the implications first.

On Wed, Jun 9, 2021 at 11:37 AM Dawid Wysakowicz <dw...@apache.org>
wrote:

> Hey,
>
> @Arvid The problem with adding the "drain/flush/stopProcessing" method to
> RichFunction is that it is not typed with the output type. At the same time
> we would most likely need a way to emit records from the method. That's
> originally thought about adding a typed interface which honestly I don't
> like that much either.
>
> On the UDF level we do not need to deprecate anything as you said. The
> close there already works as dispose on the Operator level. What we are
> suggesting is to unify that on the Operator level and deprecate the dispose
> there. @Yun I think we can already do that. We can either try to handle
> exceptions from the close in the case of a failure or just break it as it
> is a low-level, mostly internal API as Arvid said and also the migration
> would be really easy there.
>
> @Till @Arvid I am open for suggestions about the naming. I like the
> "drain" method.
>
> For now I'd go with @Piotr's proposal to add the "drain" method only to
> the SinkFunction. We think they are not immediately necessary for any of
> the other UDFs.
>
> Best,
>
> Dawid
> On 09/06/2021 11:20, Arvid Heise wrote:
>
> I have not followed the complete discussion and can't comment on the
> concepts. However, I have some ideas on the API changes:
>
> 1. If it's about adding additional life-cycle methods to UDFs, we should
> add the flush/endOfInput to RichFunction as this is the current way to
> define it. At this point, I don't see the need to add/change anything for
> UDFs. Since RichFunction does not have a dispose, do we even need to
> deprecate anything on UDF level? This would avoid having a new interface
> Flushable altogether (of which I'm not a big fan, see Piot's mail)
>
> 2. Further, I'd like to propose drain instead of flush as it would more
> align with the current nomenclature and makes the intent more obvious.
> However, that only works if there is no clash, so please double-check.
>
> 3. About changing methods on Operators: I'd say go ahead. It's
> experimental and not too hard to adjust on the user side. I also like the
> idea of beefing up ProcessFunction as a full replacement to custom
> operators but I'd keep that effort separate.
>
> On Wed, Jun 9, 2021 at 9:38 AM Till Rohrmann <tr...@apache.org> wrote:
>
>> Thanks for the lively discussion everyone. I have to admit that I am not
>> really convinced that we should call the interface Flushable and the
>> method
>> flush. The problem is that this method should in the first place tell the
>> operator that it should stop processing and flush all buffered data. The
>> method "flush" alone does not convey this contract very well. Maybe a more
>> explicit name like stopProcessingAndFlush (maybe only stopProcessing)
>> would
>> be better. Moreover, from the OutputStream.flush method, I would expect
>> that I can call this method multiple times w/o changing the state of the
>> stream. This is not the case here.
>>
>> Given that the stop processing and flush all results is such an essential
>> lifecycle method of an operator/UDF, I am not sure whether we should offer
>> it as an optional interface users can implement. The problem I see is that
>> users are not aware of it when writing their own operators/UDFs. Making it
>> part of the actual interfaces makes it more explicit and easier to
>> discover. Maybe there is a way of adding it together with a default
>> implementation, deprecating the other methods, and then eventually
>> removing
>> the old methods. The last step will break APIs, though :-(
>>
>> Cheers,
>> Till
>>
>> On Tue, Jun 8, 2021 at 6:27 PM Piotr Nowojski <pn...@apache.org>
>> wrote:
>>
>> > Hi,
>> >
>> > Thanks for resuming this discussion. I think +1 for the proposal of
>> > dropping (deprecating) `dispose()`, and adding `flush()` to the
>> > `StreamOperator`/udfs. Semantically it would be more like new `close()`
>> is
>> > an equivalent of old `dispose()`. Old `close()` is an equivalent of new
>> > `flush() + close()`. I think it provides a relatively painless migration
>> > path (could we write down this migration?).
>> >
>> > However I have some doubts about the Flushable<T> interface. First of
>> all,
>> > it wouldn't work for sinks - sinks have no output. Secondly, I don't
>> like
>> > that it opens a possibility for problems like this (incompatible output
>> > types):
>> > ```
>> > public class MyMap implements MapFunction<String, Long>,
>> Flushable<Double>
>> > { ...}
>> > ```
>> >
>> > Also after a quick offline discussion with Dawid, I'm not sure anymore
>> to
>> > which UDFs it actually makes sense to add `flush`, as most of them
>> > shouldn't buffer any data. Apart from Sinks, it's usually an operator
>> that
>> > is buffering the data (that holds true for AsyncFunction,
>> ReduceFunction,
>> > AggregateFunction, MapFunction, FilterFunction, ...). For those
>> functions
>> > it's difficult to buffer any data, as they have no means to control
>> when to
>> > emit this data. One notable exception might be (Co)ProcessFunction, as
>> it
>> > can register timers on it's own. In that case I would propose to do the
>> > following thing:
>> > 1. Add `flush() {}` to `Sink` function (FLIP-143 Writer interface
>> already
>> > has flush capabilities)
>> > 2. Maybe add `flush(Collector<O>)` to `(Co)ProcessFunction`, but maybe
>> we
>> > can postpone it
>> > 3. Leave other functions alone.
>> >
>> > After all, we could add `flush()` to other functions in the future if we
>> > really find a good motivating example to do so.
>> >
>> > About 2. Dawid is pitching an idea to convert `ProcessFunction` into a
>> > proper `Public` API that would replace StreamOperator. We could change
>> > `StreamOperator` to be purely `@Internal` class/interface, and add the
>> > missing functionality to the `ProcessFunction` (InputSelectable,
>> > BoundedInput, MailboxExecutor). With this, adding `flush()` to
>> > `ProcessFunction` would make a lot of sense. But maybe that should be a
>> > story for another day?
>> >
>> > Best,
>> > Piotrek
>> >
>> > pt., 4 cze 2021 o 10:36 Yun Gao <yu...@aliyun.com> napisał(a):
>> >
>> >> Hi all,
>> >>
>> >> Very thanks @Dawid for resuming the discussion and very thanks @Till
>> for
>> >> the summary ! (and very sorry for I missed the mail and do not response
>> >> in time...)
>> >>
>> >> I also agree with that we could consider the global commits latter
>> >> separately after we have addressed the final checkpoints, and also
>> other
>> >> points as Till summarized.
>> >> Currently the only case that have used the cascade commit is the Table
>> >> FileSystem and Hive connectors. I checked the code and found currently
>> they
>> >> will commit the
>> >> last piece of data directly  in endOfInput(). Although this might emit
>> >> repeat records if there are failover during job finishing, it avoids
>> >> emitting the records in the
>> >> notifyCheckpointComplete() after endOfInput(), thus the modification to
>> >> the operator lifecycle in final checkpoints would cause compatibility
>> >> problem for these connectors,
>> >> thus we do not need to modify them at the first place.
>> >>
>> >> 2. Regarding the operator lifecycle, I also agree with the proposed
>> >> changes. To sum up, I think the operator lifecycle would become
>> >>
>> >> endOfInput(1)
>> >> ...
>> >> endOfInput(n)
>> >> flush() --> call UDF's flush method
>> >> if some operator requires final checkpoints
>> >>     snapshotState()
>> >>     notifyCheckpointComplete()
>> >> end if
>> >> close() --> call UDF's close method
>> >>
>> >> Since currently the close() is only called in normal finish and
>> dispose()
>> >> will be called in both failover and normal case, for compatibility, I
>> think
>> >> we may
>> >> have to postpone the change to a single close() method to version 2.0 ?
>> >>
>> >> 3. Regarding the name and position of flush() method, I also agree with
>> >> that we will need a separate method to mark the termination of the
>> whole
>> >> stream for
>> >> multiple-input streams. Would it be also ok if we have some
>> modification
>> >> to the current BoundedXXInput interfaces to
>> >>
>> >> interface BoundedInput {
>> >>     void endInput() // marks the end of the whole streams, as flush()
>> >> does.
>> >> }
>> >>
>> >> @deprecated // In the future we could remove this interface
>> >> interface BoundedOneInput extends BoundedInput {}
>> >>
>> >> interface BoundedMultiInput extends BoundedInput {
>> >>       void endInput(int i);
>> >>
>> >>       default void endInput() {} // For compatibility
>> >> }
>> >>
>> >> If operator/UDF does not care about the end of a single input, then it
>> >> could directly implement the BoundedInput interface. The possible
>> >> benefit to me is that we might be able to keep only one concept for
>> >> marking the end of stream, especially for the operators with only
>> >> one input.
>> >>
>> >> Very thanks for all the deep insights and discussions!
>> >>
>> >> Best,
>> >> Yun
>> >>
>> >> ------------------------------------------------------------------
>> >> From:Dawid Wysakowicz <dw...@apache.org>
>> >> Send Time:2021 Jun. 3 (Thu.) 21:21
>> >> To:dev <de...@flink.apache.org>; Till Rohrmann <tr...@apache.org>;
>> Yun
>> >> Gao <yu...@aliyun.com>
>> >> Cc:Piotr Nowojski <pn...@apache.org>; Guowei Ma <
>> guowei.mgw@gmail.com>;
>> >> Stephan Ewen <se...@apache.org>
>> >> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
>> Finished
>> >>
>> >> Hi all,
>> >>
>> >> Thanks for the very insightful discussion. I'd like to revive the
>> effort
>> >> of FLIP-147. First of all, from my side I'd like to say that I am
>> really
>> >> interested in helping that happen in the upcoming 1.14 release.
>> >>
>> >> I agree with Till that the final checkpoints and global commits are
>> >> mostly orthogonal. Similarly as Till, I'd suggest to first focus on the
>> >> final checkpoints, while just keeping in mind we should not make
>> >> assumptions that would make it impossible to implement the global
>> commits.
>> >> So far I do not see such risk from the discussion.
>> >>
>> >> Going back to the final checkpoints issue. I think the only outstanding
>> >> issue is which methods we want to use for flushing/closing both
>> operators
>> >> and UDFs just before performing the final checkpoint. As pointed out
>> to me
>> >> by Piotr, I am mentioning UDFs here as well, because we need a way for
>> >> users using the Public API to benefit from the final checkpoint (bear
>> in
>> >> mind that e.g. TwoPhaseCommitSinkFunction which is implemented by our
>> Kafka
>> >> sink operates on the UDF level). Right now RichFunction has no method
>> which
>> >> could be called just before the final checkpoint that would say
>> "flush" all
>> >> intermediate state now and prepare for the final checkpoint. I'd
>> suggest
>> >> introducing an additional interface e.g. (name to be determined)
>> >>
>> >> interface Flushable<T> {
>> >>
>> >>    void flush(Collector<T> out)
>> >>
>> >> }
>> >>
>> >> Additionally we would need to introduce a similar method on the
>> >> StreamOperator level. Currently we have two methods that are called at
>> the
>> >> end of operator lifecycle:
>> >>
>> >>    -
>> >>    - close
>> >>    - dispose
>> >>
>> >> The usage of the two methods is a bit confusing. Dispose is responsible
>> >> for closing all open resources and is supposed to be called in case of
>> a
>> >> failure. On the other hand the close is a combination of a non-existent
>> >> "flush" method we need and dispose for closing resources in case of a
>> >> successful run. I'd suggest to clear it a bit. We would introduce a
>> proper
>> >> "flush" method which would be called in case of a successful finishing
>> of
>> >> an operator. Moreover we would make "close" deal only with closing any
>> open
>> >> resources, basically taking over the role of the dispose, which we
>> would
>> >> deprecate.
>> >>
>> >> Lastly, I'd like to say why I think it is better introduce a new
>> "flush"
>> >> method instead of using the "endInput" method of BoundedOne/MultiInput.
>> >> That method is called per input, which means each operator would need
>> to
>> >> keep track of which inputs were already closed internally and react
>> >> differently if all of the inputs were closed. With an explicit "flush"
>> >> method we do not have such a problem as the input bookkeeping happens
>> on
>> >> the StreamTask level.
>> >>
>> >> Let me know what you think. I'd sync with Yun Gao and if there are no
>> >> objections we will extend the FLIP page with necessary changes.
>> >>
>> >> Best,
>> >>
>> >> Dawid
>> >>
>> >>
>> >>
>>
>

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

Posted by Dawid Wysakowicz <dw...@apache.org>.
Hey,

@Arvid The problem with adding the "drain/flush/stopProcessing" method
to RichFunction is that it is not typed with the output type. At the
same time we would most likely need a way to emit records from the
method. That's originally thought about adding a typed interface which
honestly I don't like that much either.

On the UDF level we do not need to deprecate anything as you said. The
close there already works as dispose on the Operator level. What we are
suggesting is to unify that on the Operator level and deprecate the
dispose there. @Yun I think we can already do that. We can either try to
handle exceptions from the close in the case of a failure or just break
it as it is a low-level, mostly internal API as Arvid said and also the
migration would be really easy there.

@Till @Arvid I am open for suggestions about the naming. I like the
"drain" method.

For now I'd go with @Piotr's proposal to add the "drain" method only to
the SinkFunction. We think they are not immediately necessary for any of
the other UDFs.

Best,

Dawid

On 09/06/2021 11:20, Arvid Heise wrote:
> I have not followed the complete discussion and can't comment on the
> concepts. However, I have some ideas on the API changes:
>
> 1. If it's about adding additional life-cycle methods to UDFs, we
> should add the flush/endOfInput to RichFunction as this is the current
> way to define it. At this point, I don't see the need to add/change
> anything for UDFs. Since RichFunction does not have a dispose, do we
> even need to deprecate anything on UDF level? This would avoid having
> a new interface Flushable altogether (of which I'm not a big fan, see
> Piot's mail)
>
> 2. Further, I'd like to propose drain instead of flush as it would
> more align with the current nomenclature and makes the intent more
> obvious. However, that only works if there is no clash, so please
> double-check.
>
> 3. About changing methods on Operators: I'd say go ahead. It's
> experimental and not too hard to adjust on the user side. I also like
> the idea of beefing up ProcessFunction as a full replacement to custom
> operators but I'd keep that effort separate.
>
> On Wed, Jun 9, 2021 at 9:38 AM Till Rohrmann <trohrmann@apache.org
> <ma...@apache.org>> wrote:
>
>     Thanks for the lively discussion everyone. I have to admit that I
>     am not
>     really convinced that we should call the interface Flushable and
>     the method
>     flush. The problem is that this method should in the first place
>     tell the
>     operator that it should stop processing and flush all buffered
>     data. The
>     method "flush" alone does not convey this contract very well.
>     Maybe a more
>     explicit name like stopProcessingAndFlush (maybe only
>     stopProcessing) would
>     be better. Moreover, from the OutputStream.flush method, I would
>     expect
>     that I can call this method multiple times w/o changing the state
>     of the
>     stream. This is not the case here.
>
>     Given that the stop processing and flush all results is such an
>     essential
>     lifecycle method of an operator/UDF, I am not sure whether we
>     should offer
>     it as an optional interface users can implement. The problem I see
>     is that
>     users are not aware of it when writing their own operators/UDFs.
>     Making it
>     part of the actual interfaces makes it more explicit and easier to
>     discover. Maybe there is a way of adding it together with a default
>     implementation, deprecating the other methods, and then eventually
>     removing
>     the old methods. The last step will break APIs, though :-(
>
>     Cheers,
>     Till
>
>     On Tue, Jun 8, 2021 at 6:27 PM Piotr Nowojski
>     <pnowojski@apache.org <ma...@apache.org>> wrote:
>
>     > Hi,
>     >
>     > Thanks for resuming this discussion. I think +1 for the proposal of
>     > dropping (deprecating) `dispose()`, and adding `flush()` to the
>     > `StreamOperator`/udfs. Semantically it would be more like new
>     `close()` is
>     > an equivalent of old `dispose()`. Old `close()` is an equivalent
>     of new
>     > `flush() + close()`. I think it provides a relatively painless
>     migration
>     > path (could we write down this migration?).
>     >
>     > However I have some doubts about the Flushable<T> interface.
>     First of all,
>     > it wouldn't work for sinks - sinks have no output. Secondly, I
>     don't like
>     > that it opens a possibility for problems like this (incompatible
>     output
>     > types):
>     > ```
>     > public class MyMap implements MapFunction<String, Long>,
>     Flushable<Double>
>     > { ...}
>     > ```
>     >
>     > Also after a quick offline discussion with Dawid, I'm not sure
>     anymore to
>     > which UDFs it actually makes sense to add `flush`, as most of them
>     > shouldn't buffer any data. Apart from Sinks, it's usually an
>     operator that
>     > is buffering the data (that holds true for AsyncFunction,
>     ReduceFunction,
>     > AggregateFunction, MapFunction, FilterFunction, ...). For those
>     functions
>     > it's difficult to buffer any data, as they have no means to
>     control when to
>     > emit this data. One notable exception might be
>     (Co)ProcessFunction, as it
>     > can register timers on it's own. In that case I would propose to
>     do the
>     > following thing:
>     > 1. Add `flush() {}` to `Sink` function (FLIP-143 Writer
>     interface already
>     > has flush capabilities)
>     > 2. Maybe add `flush(Collector<O>)` to `(Co)ProcessFunction`, but
>     maybe we
>     > can postpone it
>     > 3. Leave other functions alone.
>     >
>     > After all, we could add `flush()` to other functions in the
>     future if we
>     > really find a good motivating example to do so.
>     >
>     > About 2. Dawid is pitching an idea to convert `ProcessFunction`
>     into a
>     > proper `Public` API that would replace StreamOperator. We could
>     change
>     > `StreamOperator` to be purely `@Internal` class/interface, and
>     add the
>     > missing functionality to the `ProcessFunction` (InputSelectable,
>     > BoundedInput, MailboxExecutor). With this, adding `flush()` to
>     > `ProcessFunction` would make a lot of sense. But maybe that
>     should be a
>     > story for another day?
>     >
>     > Best,
>     > Piotrek
>     >
>     > pt., 4 cze 2021 o 10:36 Yun Gao <yungao.gy@aliyun.com
>     <ma...@aliyun.com>> napisał(a):
>     >
>     >> Hi all,
>     >>
>     >> Very thanks @Dawid for resuming the discussion and very thanks
>     @Till for
>     >> the summary ! (and very sorry for I missed the mail and do not
>     response
>     >> in time...)
>     >>
>     >> I also agree with that we could consider the global commits latter
>     >> separately after we have addressed the final checkpoints, and
>     also other
>     >> points as Till summarized.
>     >> Currently the only case that have used the cascade commit is
>     the Table
>     >> FileSystem and Hive connectors. I checked the code and found
>     currently they
>     >> will commit the
>     >> last piece of data directly  in endOfInput(). Although this
>     might emit
>     >> repeat records if there are failover during job finishing, it
>     avoids
>     >> emitting the records in the
>     >> notifyCheckpointComplete() after endOfInput(), thus the
>     modification to
>     >> the operator lifecycle in final checkpoints would cause
>     compatibility
>     >> problem for these connectors,
>     >> thus we do not need to modify them at the first place.
>     >>
>     >> 2. Regarding the operator lifecycle, I also agree with the proposed
>     >> changes. To sum up, I think the operator lifecycle would become
>     >>
>     >> endOfInput(1)
>     >> ...
>     >> endOfInput(n)
>     >> flush() --> call UDF's flush method
>     >> if some operator requires final checkpoints
>     >>     snapshotState()
>     >>     notifyCheckpointComplete()
>     >> end if
>     >> close() --> call UDF's close method
>     >>
>     >> Since currently the close() is only called in normal finish and
>     dispose()
>     >> will be called in both failover and normal case, for
>     compatibility, I think
>     >> we may
>     >> have to postpone the change to a single close() method to
>     version 2.0 ?
>     >>
>     >> 3. Regarding the name and position of flush() method, I also
>     agree with
>     >> that we will need a separate method to mark the termination of
>     the whole
>     >> stream for
>     >> multiple-input streams. Would it be also ok if we have some
>     modification
>     >> to the current BoundedXXInput interfaces to
>     >>
>     >> interface BoundedInput {
>     >>     void endInput() // marks the end of the whole streams, as
>     flush()
>     >> does.
>     >> }
>     >>
>     >> @deprecated // In the future we could remove this interface
>     >> interface BoundedOneInput extends BoundedInput {}
>     >>
>     >> interface BoundedMultiInput extends BoundedInput {
>     >>       void endInput(int i);
>     >>
>     >>       default void endInput() {} // For compatibility
>     >> }
>     >>
>     >> If operator/UDF does not care about the end of a single input,
>     then it
>     >> could directly implement the BoundedInput interface. The possible
>     >> benefit to me is that we might be able to keep only one concept for
>     >> marking the end of stream, especially for the operators with only
>     >> one input.
>     >>
>     >> Very thanks for all the deep insights and discussions!
>     >>
>     >> Best,
>     >> Yun
>     >>
>     >> ------------------------------------------------------------------
>     >> From:Dawid Wysakowicz <dwysakowicz@apache.org
>     <ma...@apache.org>>
>     >> Send Time:2021 Jun. 3 (Thu.) 21:21
>     >> To:dev <dev@flink.apache.org <ma...@flink.apache.org>>;
>     Till Rohrmann <trohrmann@apache.org
>     <ma...@apache.org>>; Yun
>     >> Gao <yungao.gy@aliyun.com <ma...@aliyun.com>>
>     >> Cc:Piotr Nowojski <pnowojski@apache.org
>     <ma...@apache.org>>; Guowei Ma <guowei.mgw@gmail.com
>     <ma...@gmail.com>>;
>     >> Stephan Ewen <sewen@apache.org <ma...@apache.org>>
>     >> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
>     Finished
>     >>
>     >> Hi all,
>     >>
>     >> Thanks for the very insightful discussion. I'd like to revive
>     the effort
>     >> of FLIP-147. First of all, from my side I'd like to say that I
>     am really
>     >> interested in helping that happen in the upcoming 1.14 release.
>     >>
>     >> I agree with Till that the final checkpoints and global commits are
>     >> mostly orthogonal. Similarly as Till, I'd suggest to first
>     focus on the
>     >> final checkpoints, while just keeping in mind we should not make
>     >> assumptions that would make it impossible to implement the
>     global commits.
>     >> So far I do not see such risk from the discussion.
>     >>
>     >> Going back to the final checkpoints issue. I think the only
>     outstanding
>     >> issue is which methods we want to use for flushing/closing both
>     operators
>     >> and UDFs just before performing the final checkpoint. As
>     pointed out to me
>     >> by Piotr, I am mentioning UDFs here as well, because we need a
>     way for
>     >> users using the Public API to benefit from the final checkpoint
>     (bear in
>     >> mind that e.g. TwoPhaseCommitSinkFunction which is implemented
>     by our Kafka
>     >> sink operates on the UDF level). Right now RichFunction has no
>     method which
>     >> could be called just before the final checkpoint that would say
>     "flush" all
>     >> intermediate state now and prepare for the final checkpoint.
>     I'd suggest
>     >> introducing an additional interface e.g. (name to be determined)
>     >>
>     >> interface Flushable<T> {
>     >>
>     >>    void flush(Collector<T> out)
>     >>
>     >> }
>     >>
>     >> Additionally we would need to introduce a similar method on the
>     >> StreamOperator level. Currently we have two methods that are
>     called at the
>     >> end of operator lifecycle:
>     >>
>     >>    -
>     >>    - close
>     >>    - dispose
>     >>
>     >> The usage of the two methods is a bit confusing. Dispose is
>     responsible
>     >> for closing all open resources and is supposed to be called in
>     case of a
>     >> failure. On the other hand the close is a combination of a
>     non-existent
>     >> "flush" method we need and dispose for closing resources in
>     case of a
>     >> successful run. I'd suggest to clear it a bit. We would
>     introduce a proper
>     >> "flush" method which would be called in case of a successful
>     finishing of
>     >> an operator. Moreover we would make "close" deal only with
>     closing any open
>     >> resources, basically taking over the role of the dispose, which
>     we would
>     >> deprecate.
>     >>
>     >> Lastly, I'd like to say why I think it is better introduce a
>     new "flush"
>     >> method instead of using the "endInput" method of
>     BoundedOne/MultiInput.
>     >> That method is called per input, which means each operator
>     would need to
>     >> keep track of which inputs were already closed internally and react
>     >> differently if all of the inputs were closed. With an explicit
>     "flush"
>     >> method we do not have such a problem as the input bookkeeping
>     happens on
>     >> the StreamTask level.
>     >>
>     >> Let me know what you think. I'd sync with Yun Gao and if there
>     are no
>     >> objections we will extend the FLIP page with necessary changes.
>     >>
>     >> Best,
>     >>
>     >> Dawid
>     >>
>     >>
>     >>
>

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

Posted by Arvid Heise <ar...@apache.org>.
I have not followed the complete discussion and can't comment on the
concepts. However, I have some ideas on the API changes:

1. If it's about adding additional life-cycle methods to UDFs, we should
add the flush/endOfInput to RichFunction as this is the current way to
define it. At this point, I don't see the need to add/change anything for
UDFs. Since RichFunction does not have a dispose, do we even need to
deprecate anything on UDF level? This would avoid having a new interface
Flushable altogether (of which I'm not a big fan, see Piot's mail)

2. Further, I'd like to propose drain instead of flush as it would more
align with the current nomenclature and makes the intent more obvious.
However, that only works if there is no clash, so please double-check.

3. About changing methods on Operators: I'd say go ahead. It's experimental
and not too hard to adjust on the user side. I also like the idea of
beefing up ProcessFunction as a full replacement to custom operators but
I'd keep that effort separate.

On Wed, Jun 9, 2021 at 9:38 AM Till Rohrmann <tr...@apache.org> wrote:

> Thanks for the lively discussion everyone. I have to admit that I am not
> really convinced that we should call the interface Flushable and the method
> flush. The problem is that this method should in the first place tell the
> operator that it should stop processing and flush all buffered data. The
> method "flush" alone does not convey this contract very well. Maybe a more
> explicit name like stopProcessingAndFlush (maybe only stopProcessing) would
> be better. Moreover, from the OutputStream.flush method, I would expect
> that I can call this method multiple times w/o changing the state of the
> stream. This is not the case here.
>
> Given that the stop processing and flush all results is such an essential
> lifecycle method of an operator/UDF, I am not sure whether we should offer
> it as an optional interface users can implement. The problem I see is that
> users are not aware of it when writing their own operators/UDFs. Making it
> part of the actual interfaces makes it more explicit and easier to
> discover. Maybe there is a way of adding it together with a default
> implementation, deprecating the other methods, and then eventually removing
> the old methods. The last step will break APIs, though :-(
>
> Cheers,
> Till
>
> On Tue, Jun 8, 2021 at 6:27 PM Piotr Nowojski <pn...@apache.org>
> wrote:
>
> > Hi,
> >
> > Thanks for resuming this discussion. I think +1 for the proposal of
> > dropping (deprecating) `dispose()`, and adding `flush()` to the
> > `StreamOperator`/udfs. Semantically it would be more like new `close()`
> is
> > an equivalent of old `dispose()`. Old `close()` is an equivalent of new
> > `flush() + close()`. I think it provides a relatively painless migration
> > path (could we write down this migration?).
> >
> > However I have some doubts about the Flushable<T> interface. First of
> all,
> > it wouldn't work for sinks - sinks have no output. Secondly, I don't like
> > that it opens a possibility for problems like this (incompatible output
> > types):
> > ```
> > public class MyMap implements MapFunction<String, Long>,
> Flushable<Double>
> > { ...}
> > ```
> >
> > Also after a quick offline discussion with Dawid, I'm not sure anymore to
> > which UDFs it actually makes sense to add `flush`, as most of them
> > shouldn't buffer any data. Apart from Sinks, it's usually an operator
> that
> > is buffering the data (that holds true for AsyncFunction, ReduceFunction,
> > AggregateFunction, MapFunction, FilterFunction, ...). For those functions
> > it's difficult to buffer any data, as they have no means to control when
> to
> > emit this data. One notable exception might be (Co)ProcessFunction, as it
> > can register timers on it's own. In that case I would propose to do the
> > following thing:
> > 1. Add `flush() {}` to `Sink` function (FLIP-143 Writer interface already
> > has flush capabilities)
> > 2. Maybe add `flush(Collector<O>)` to `(Co)ProcessFunction`, but maybe we
> > can postpone it
> > 3. Leave other functions alone.
> >
> > After all, we could add `flush()` to other functions in the future if we
> > really find a good motivating example to do so.
> >
> > About 2. Dawid is pitching an idea to convert `ProcessFunction` into a
> > proper `Public` API that would replace StreamOperator. We could change
> > `StreamOperator` to be purely `@Internal` class/interface, and add the
> > missing functionality to the `ProcessFunction` (InputSelectable,
> > BoundedInput, MailboxExecutor). With this, adding `flush()` to
> > `ProcessFunction` would make a lot of sense. But maybe that should be a
> > story for another day?
> >
> > Best,
> > Piotrek
> >
> > pt., 4 cze 2021 o 10:36 Yun Gao <yu...@aliyun.com> napisał(a):
> >
> >> Hi all,
> >>
> >> Very thanks @Dawid for resuming the discussion and very thanks @Till for
> >> the summary ! (and very sorry for I missed the mail and do not response
> >> in time...)
> >>
> >> I also agree with that we could consider the global commits latter
> >> separately after we have addressed the final checkpoints, and also other
> >> points as Till summarized.
> >> Currently the only case that have used the cascade commit is the Table
> >> FileSystem and Hive connectors. I checked the code and found currently
> they
> >> will commit the
> >> last piece of data directly  in endOfInput(). Although this might emit
> >> repeat records if there are failover during job finishing, it avoids
> >> emitting the records in the
> >> notifyCheckpointComplete() after endOfInput(), thus the modification to
> >> the operator lifecycle in final checkpoints would cause compatibility
> >> problem for these connectors,
> >> thus we do not need to modify them at the first place.
> >>
> >> 2. Regarding the operator lifecycle, I also agree with the proposed
> >> changes. To sum up, I think the operator lifecycle would become
> >>
> >> endOfInput(1)
> >> ...
> >> endOfInput(n)
> >> flush() --> call UDF's flush method
> >> if some operator requires final checkpoints
> >>     snapshotState()
> >>     notifyCheckpointComplete()
> >> end if
> >> close() --> call UDF's close method
> >>
> >> Since currently the close() is only called in normal finish and
> dispose()
> >> will be called in both failover and normal case, for compatibility, I
> think
> >> we may
> >> have to postpone the change to a single close() method to version 2.0 ?
> >>
> >> 3. Regarding the name and position of flush() method, I also agree with
> >> that we will need a separate method to mark the termination of the whole
> >> stream for
> >> multiple-input streams. Would it be also ok if we have some modification
> >> to the current BoundedXXInput interfaces to
> >>
> >> interface BoundedInput {
> >>     void endInput() // marks the end of the whole streams, as flush()
> >> does.
> >> }
> >>
> >> @deprecated // In the future we could remove this interface
> >> interface BoundedOneInput extends BoundedInput {}
> >>
> >> interface BoundedMultiInput extends BoundedInput {
> >>       void endInput(int i);
> >>
> >>       default void endInput() {} // For compatibility
> >> }
> >>
> >> If operator/UDF does not care about the end of a single input, then it
> >> could directly implement the BoundedInput interface. The possible
> >> benefit to me is that we might be able to keep only one concept for
> >> marking the end of stream, especially for the operators with only
> >> one input.
> >>
> >> Very thanks for all the deep insights and discussions!
> >>
> >> Best,
> >> Yun
> >>
> >> ------------------------------------------------------------------
> >> From:Dawid Wysakowicz <dw...@apache.org>
> >> Send Time:2021 Jun. 3 (Thu.) 21:21
> >> To:dev <de...@flink.apache.org>; Till Rohrmann <tr...@apache.org>;
> Yun
> >> Gao <yu...@aliyun.com>
> >> Cc:Piotr Nowojski <pn...@apache.org>; Guowei Ma <
> guowei.mgw@gmail.com>;
> >> Stephan Ewen <se...@apache.org>
> >> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
> >>
> >> Hi all,
> >>
> >> Thanks for the very insightful discussion. I'd like to revive the effort
> >> of FLIP-147. First of all, from my side I'd like to say that I am really
> >> interested in helping that happen in the upcoming 1.14 release.
> >>
> >> I agree with Till that the final checkpoints and global commits are
> >> mostly orthogonal. Similarly as Till, I'd suggest to first focus on the
> >> final checkpoints, while just keeping in mind we should not make
> >> assumptions that would make it impossible to implement the global
> commits.
> >> So far I do not see such risk from the discussion.
> >>
> >> Going back to the final checkpoints issue. I think the only outstanding
> >> issue is which methods we want to use for flushing/closing both
> operators
> >> and UDFs just before performing the final checkpoint. As pointed out to
> me
> >> by Piotr, I am mentioning UDFs here as well, because we need a way for
> >> users using the Public API to benefit from the final checkpoint (bear in
> >> mind that e.g. TwoPhaseCommitSinkFunction which is implemented by our
> Kafka
> >> sink operates on the UDF level). Right now RichFunction has no method
> which
> >> could be called just before the final checkpoint that would say "flush"
> all
> >> intermediate state now and prepare for the final checkpoint. I'd suggest
> >> introducing an additional interface e.g. (name to be determined)
> >>
> >> interface Flushable<T> {
> >>
> >>    void flush(Collector<T> out)
> >>
> >> }
> >>
> >> Additionally we would need to introduce a similar method on the
> >> StreamOperator level. Currently we have two methods that are called at
> the
> >> end of operator lifecycle:
> >>
> >>    -
> >>    - close
> >>    - dispose
> >>
> >> The usage of the two methods is a bit confusing. Dispose is responsible
> >> for closing all open resources and is supposed to be called in case of a
> >> failure. On the other hand the close is a combination of a non-existent
> >> "flush" method we need and dispose for closing resources in case of a
> >> successful run. I'd suggest to clear it a bit. We would introduce a
> proper
> >> "flush" method which would be called in case of a successful finishing
> of
> >> an operator. Moreover we would make "close" deal only with closing any
> open
> >> resources, basically taking over the role of the dispose, which we would
> >> deprecate.
> >>
> >> Lastly, I'd like to say why I think it is better introduce a new "flush"
> >> method instead of using the "endInput" method of BoundedOne/MultiInput.
> >> That method is called per input, which means each operator would need to
> >> keep track of which inputs were already closed internally and react
> >> differently if all of the inputs were closed. With an explicit "flush"
> >> method we do not have such a problem as the input bookkeeping happens on
> >> the StreamTask level.
> >>
> >> Let me know what you think. I'd sync with Yun Gao and if there are no
> >> objections we will extend the FLIP page with necessary changes.
> >>
> >> Best,
> >>
> >> Dawid
> >>
> >>
> >>
>

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

Posted by Till Rohrmann <tr...@apache.org>.
Thanks for the lively discussion everyone. I have to admit that I am not
really convinced that we should call the interface Flushable and the method
flush. The problem is that this method should in the first place tell the
operator that it should stop processing and flush all buffered data. The
method "flush" alone does not convey this contract very well. Maybe a more
explicit name like stopProcessingAndFlush (maybe only stopProcessing) would
be better. Moreover, from the OutputStream.flush method, I would expect
that I can call this method multiple times w/o changing the state of the
stream. This is not the case here.

Given that the stop processing and flush all results is such an essential
lifecycle method of an operator/UDF, I am not sure whether we should offer
it as an optional interface users can implement. The problem I see is that
users are not aware of it when writing their own operators/UDFs. Making it
part of the actual interfaces makes it more explicit and easier to
discover. Maybe there is a way of adding it together with a default
implementation, deprecating the other methods, and then eventually removing
the old methods. The last step will break APIs, though :-(

Cheers,
Till

On Tue, Jun 8, 2021 at 6:27 PM Piotr Nowojski <pn...@apache.org> wrote:

> Hi,
>
> Thanks for resuming this discussion. I think +1 for the proposal of
> dropping (deprecating) `dispose()`, and adding `flush()` to the
> `StreamOperator`/udfs. Semantically it would be more like new `close()` is
> an equivalent of old `dispose()`. Old `close()` is an equivalent of new
> `flush() + close()`. I think it provides a relatively painless migration
> path (could we write down this migration?).
>
> However I have some doubts about the Flushable<T> interface. First of all,
> it wouldn't work for sinks - sinks have no output. Secondly, I don't like
> that it opens a possibility for problems like this (incompatible output
> types):
> ```
> public class MyMap implements MapFunction<String, Long>, Flushable<Double>
> { ...}
> ```
>
> Also after a quick offline discussion with Dawid, I'm not sure anymore to
> which UDFs it actually makes sense to add `flush`, as most of them
> shouldn't buffer any data. Apart from Sinks, it's usually an operator that
> is buffering the data (that holds true for AsyncFunction, ReduceFunction,
> AggregateFunction, MapFunction, FilterFunction, ...). For those functions
> it's difficult to buffer any data, as they have no means to control when to
> emit this data. One notable exception might be (Co)ProcessFunction, as it
> can register timers on it's own. In that case I would propose to do the
> following thing:
> 1. Add `flush() {}` to `Sink` function (FLIP-143 Writer interface already
> has flush capabilities)
> 2. Maybe add `flush(Collector<O>)` to `(Co)ProcessFunction`, but maybe we
> can postpone it
> 3. Leave other functions alone.
>
> After all, we could add `flush()` to other functions in the future if we
> really find a good motivating example to do so.
>
> About 2. Dawid is pitching an idea to convert `ProcessFunction` into a
> proper `Public` API that would replace StreamOperator. We could change
> `StreamOperator` to be purely `@Internal` class/interface, and add the
> missing functionality to the `ProcessFunction` (InputSelectable,
> BoundedInput, MailboxExecutor). With this, adding `flush()` to
> `ProcessFunction` would make a lot of sense. But maybe that should be a
> story for another day?
>
> Best,
> Piotrek
>
> pt., 4 cze 2021 o 10:36 Yun Gao <yu...@aliyun.com> napisał(a):
>
>> Hi all,
>>
>> Very thanks @Dawid for resuming the discussion and very thanks @Till for
>> the summary ! (and very sorry for I missed the mail and do not response
>> in time...)
>>
>> I also agree with that we could consider the global commits latter
>> separately after we have addressed the final checkpoints, and also other
>> points as Till summarized.
>> Currently the only case that have used the cascade commit is the Table
>> FileSystem and Hive connectors. I checked the code and found currently they
>> will commit the
>> last piece of data directly  in endOfInput(). Although this might emit
>> repeat records if there are failover during job finishing, it avoids
>> emitting the records in the
>> notifyCheckpointComplete() after endOfInput(), thus the modification to
>> the operator lifecycle in final checkpoints would cause compatibility
>> problem for these connectors,
>> thus we do not need to modify them at the first place.
>>
>> 2. Regarding the operator lifecycle, I also agree with the proposed
>> changes. To sum up, I think the operator lifecycle would become
>>
>> endOfInput(1)
>> ...
>> endOfInput(n)
>> flush() --> call UDF's flush method
>> if some operator requires final checkpoints
>>     snapshotState()
>>     notifyCheckpointComplete()
>> end if
>> close() --> call UDF's close method
>>
>> Since currently the close() is only called in normal finish and dispose()
>> will be called in both failover and normal case, for compatibility, I think
>> we may
>> have to postpone the change to a single close() method to version 2.0 ?
>>
>> 3. Regarding the name and position of flush() method, I also agree with
>> that we will need a separate method to mark the termination of the whole
>> stream for
>> multiple-input streams. Would it be also ok if we have some modification
>> to the current BoundedXXInput interfaces to
>>
>> interface BoundedInput {
>>     void endInput() // marks the end of the whole streams, as flush()
>> does.
>> }
>>
>> @deprecated // In the future we could remove this interface
>> interface BoundedOneInput extends BoundedInput {}
>>
>> interface BoundedMultiInput extends BoundedInput {
>>       void endInput(int i);
>>
>>       default void endInput() {} // For compatibility
>> }
>>
>> If operator/UDF does not care about the end of a single input, then it
>> could directly implement the BoundedInput interface. The possible
>> benefit to me is that we might be able to keep only one concept for
>> marking the end of stream, especially for the operators with only
>> one input.
>>
>> Very thanks for all the deep insights and discussions!
>>
>> Best,
>> Yun
>>
>> ------------------------------------------------------------------
>> From:Dawid Wysakowicz <dw...@apache.org>
>> Send Time:2021 Jun. 3 (Thu.) 21:21
>> To:dev <de...@flink.apache.org>; Till Rohrmann <tr...@apache.org>; Yun
>> Gao <yu...@aliyun.com>
>> Cc:Piotr Nowojski <pn...@apache.org>; Guowei Ma <gu...@gmail.com>;
>> Stephan Ewen <se...@apache.org>
>> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>>
>> Hi all,
>>
>> Thanks for the very insightful discussion. I'd like to revive the effort
>> of FLIP-147. First of all, from my side I'd like to say that I am really
>> interested in helping that happen in the upcoming 1.14 release.
>>
>> I agree with Till that the final checkpoints and global commits are
>> mostly orthogonal. Similarly as Till, I'd suggest to first focus on the
>> final checkpoints, while just keeping in mind we should not make
>> assumptions that would make it impossible to implement the global commits.
>> So far I do not see such risk from the discussion.
>>
>> Going back to the final checkpoints issue. I think the only outstanding
>> issue is which methods we want to use for flushing/closing both operators
>> and UDFs just before performing the final checkpoint. As pointed out to me
>> by Piotr, I am mentioning UDFs here as well, because we need a way for
>> users using the Public API to benefit from the final checkpoint (bear in
>> mind that e.g. TwoPhaseCommitSinkFunction which is implemented by our Kafka
>> sink operates on the UDF level). Right now RichFunction has no method which
>> could be called just before the final checkpoint that would say "flush" all
>> intermediate state now and prepare for the final checkpoint. I'd suggest
>> introducing an additional interface e.g. (name to be determined)
>>
>> interface Flushable<T> {
>>
>>    void flush(Collector<T> out)
>>
>> }
>>
>> Additionally we would need to introduce a similar method on the
>> StreamOperator level. Currently we have two methods that are called at the
>> end of operator lifecycle:
>>
>>    -
>>    - close
>>    - dispose
>>
>> The usage of the two methods is a bit confusing. Dispose is responsible
>> for closing all open resources and is supposed to be called in case of a
>> failure. On the other hand the close is a combination of a non-existent
>> "flush" method we need and dispose for closing resources in case of a
>> successful run. I'd suggest to clear it a bit. We would introduce a proper
>> "flush" method which would be called in case of a successful finishing of
>> an operator. Moreover we would make "close" deal only with closing any open
>> resources, basically taking over the role of the dispose, which we would
>> deprecate.
>>
>> Lastly, I'd like to say why I think it is better introduce a new "flush"
>> method instead of using the "endInput" method of BoundedOne/MultiInput.
>> That method is called per input, which means each operator would need to
>> keep track of which inputs were already closed internally and react
>> differently if all of the inputs were closed. With an explicit "flush"
>> method we do not have such a problem as the input bookkeeping happens on
>> the StreamTask level.
>>
>> Let me know what you think. I'd sync with Yun Gao and if there are no
>> objections we will extend the FLIP page with necessary changes.
>>
>> Best,
>>
>> Dawid
>>
>>
>>

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

Posted by Piotr Nowojski <pn...@apache.org>.
Hi,

Thanks for resuming this discussion. I think +1 for the proposal of
dropping (deprecating) `dispose()`, and adding `flush()` to the
`StreamOperator`/udfs. Semantically it would be more like new `close()` is
an equivalent of old `dispose()`. Old `close()` is an equivalent of new
`flush() + close()`. I think it provides a relatively painless migration
path (could we write down this migration?).

However I have some doubts about the Flushable<T> interface. First of all,
it wouldn't work for sinks - sinks have no output. Secondly, I don't like
that it opens a possibility for problems like this (incompatible output
types):
```
public class MyMap implements MapFunction<String, Long>, Flushable<Double>
{ ...}
```

Also after a quick offline discussion with Dawid, I'm not sure anymore to
which UDFs it actually makes sense to add `flush`, as most of them
shouldn't buffer any data. Apart from Sinks, it's usually an operator that
is buffering the data (that holds true for AsyncFunction, ReduceFunction,
AggregateFunction, MapFunction, FilterFunction, ...). For those functions
it's difficult to buffer any data, as they have no means to control when to
emit this data. One notable exception might be (Co)ProcessFunction, as it
can register timers on it's own. In that case I would propose to do the
following thing:
1. Add `flush() {}` to `Sink` function (FLIP-143 Writer interface already
has flush capabilities)
2. Maybe add `flush(Collector<O>)` to `(Co)ProcessFunction`, but maybe we
can postpone it
3. Leave other functions alone.

After all, we could add `flush()` to other functions in the future if we
really find a good motivating example to do so.

About 2. Dawid is pitching an idea to convert `ProcessFunction` into a
proper `Public` API that would replace StreamOperator. We could change
`StreamOperator` to be purely `@Internal` class/interface, and add the
missing functionality to the `ProcessFunction` (InputSelectable,
BoundedInput, MailboxExecutor). With this, adding `flush()` to
`ProcessFunction` would make a lot of sense. But maybe that should be a
story for another day?

Best,
Piotrek

pt., 4 cze 2021 o 10:36 Yun Gao <yu...@aliyun.com> napisał(a):

> Hi all,
>
> Very thanks @Dawid for resuming the discussion and very thanks @Till for
> the summary ! (and very sorry for I missed the mail and do not response
> in time...)
>
> I also agree with that we could consider the global commits latter
> separately after we have addressed the final checkpoints, and also other
> points as Till summarized.
> Currently the only case that have used the cascade commit is the Table
> FileSystem and Hive connectors. I checked the code and found currently they
> will commit the
> last piece of data directly  in endOfInput(). Although this might emit
> repeat records if there are failover during job finishing, it avoids
> emitting the records in the
> notifyCheckpointComplete() after endOfInput(), thus the modification to
> the operator lifecycle in final checkpoints would cause compatibility
> problem for these connectors,
> thus we do not need to modify them at the first place.
>
> 2. Regarding the operator lifecycle, I also agree with the proposed
> changes. To sum up, I think the operator lifecycle would become
>
> endOfInput(1)
> ...
> endOfInput(n)
> flush() --> call UDF's flush method
> if some operator requires final checkpoints
>     snapshotState()
>     notifyCheckpointComplete()
> end if
> close() --> call UDF's close method
>
> Since currently the close() is only called in normal finish and dispose()
> will be called in both failover and normal case, for compatibility, I think
> we may
> have to postpone the change to a single close() method to version 2.0 ?
>
> 3. Regarding the name and position of flush() method, I also agree with
> that we will need a separate method to mark the termination of the whole
> stream for
> multiple-input streams. Would it be also ok if we have some modification
> to the current BoundedXXInput interfaces to
>
> interface BoundedInput {
>     void endInput() // marks the end of the whole streams, as flush()
> does.
> }
>
> @deprecated // In the future we could remove this interface
> interface BoundedOneInput extends BoundedInput {}
>
> interface BoundedMultiInput extends BoundedInput {
>       void endInput(int i);
>
>       default void endInput() {} // For compatibility
> }
>
> If operator/UDF does not care about the end of a single input, then it
> could directly implement the BoundedInput interface. The possible
> benefit to me is that we might be able to keep only one concept for
> marking the end of stream, especially for the operators with only
> one input.
>
> Very thanks for all the deep insights and discussions!
>
> Best,
> Yun
>
> ------------------------------------------------------------------
> From:Dawid Wysakowicz <dw...@apache.org>
> Send Time:2021 Jun. 3 (Thu.) 21:21
> To:dev <de...@flink.apache.org>; Till Rohrmann <tr...@apache.org>; Yun
> Gao <yu...@aliyun.com>
> Cc:Piotr Nowojski <pn...@apache.org>; Guowei Ma <gu...@gmail.com>;
> Stephan Ewen <se...@apache.org>
> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
> Hi all,
>
> Thanks for the very insightful discussion. I'd like to revive the effort
> of FLIP-147. First of all, from my side I'd like to say that I am really
> interested in helping that happen in the upcoming 1.14 release.
>
> I agree with Till that the final checkpoints and global commits are mostly
> orthogonal. Similarly as Till, I'd suggest to first focus on the final
> checkpoints, while just keeping in mind we should not make assumptions that
> would make it impossible to implement the global commits. So far I do not
> see such risk from the discussion.
>
> Going back to the final checkpoints issue. I think the only outstanding
> issue is which methods we want to use for flushing/closing both operators
> and UDFs just before performing the final checkpoint. As pointed out to me
> by Piotr, I am mentioning UDFs here as well, because we need a way for
> users using the Public API to benefit from the final checkpoint (bear in
> mind that e.g. TwoPhaseCommitSinkFunction which is implemented by our Kafka
> sink operates on the UDF level). Right now RichFunction has no method which
> could be called just before the final checkpoint that would say "flush" all
> intermediate state now and prepare for the final checkpoint. I'd suggest
> introducing an additional interface e.g. (name to be determined)
>
> interface Flushable<T> {
>
>    void flush(Collector<T> out)
>
> }
>
> Additionally we would need to introduce a similar method on the
> StreamOperator level. Currently we have two methods that are called at the
> end of operator lifecycle:
>
>    -
>    - close
>    - dispose
>
> The usage of the two methods is a bit confusing. Dispose is responsible
> for closing all open resources and is supposed to be called in case of a
> failure. On the other hand the close is a combination of a non-existent
> "flush" method we need and dispose for closing resources in case of a
> successful run. I'd suggest to clear it a bit. We would introduce a proper
> "flush" method which would be called in case of a successful finishing of
> an operator. Moreover we would make "close" deal only with closing any open
> resources, basically taking over the role of the dispose, which we would
> deprecate.
>
> Lastly, I'd like to say why I think it is better introduce a new "flush"
> method instead of using the "endInput" method of BoundedOne/MultiInput.
> That method is called per input, which means each operator would need to
> keep track of which inputs were already closed internally and react
> differently if all of the inputs were closed. With an explicit "flush"
> method we do not have such a problem as the input bookkeeping happens on
> the StreamTask level.
>
> Let me know what you think. I'd sync with Yun Gao and if there are no
> objections we will extend the FLIP page with necessary changes.
>
> Best,
>
> Dawid
>
>
>

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi all,

Very thanks @Dawid for resuming the discussion and very thanks @Till for the summary ! (and very sorry for I missed the mail and do not response in time...)

I also agree with that we could consider the global commits latter separately after we have addressed the final checkpoints, and also other points as Till summarized. 
Currently the only case that have used the cascade commit is the Table FileSystem and Hive connectors. I checked the code and found currently they will commit the 
last piece of data directly  in endOfInput(). Although this might emit repeat records if there are failover during job finishing, it avoids emitting the records in the 
notifyCheckpointComplete() after endOfInput(), thus the modification to the operator lifecycle in final checkpoints would cause compatibility problem for these connectors, 
thus we do not need to modify them at the first place. 

2. Regarding the operator lifecycle, I also agree with the proposed changes. To sum up, I think the operator lifecycle would become 

endOfInput(1)
...
endOfInput(n)
flush() --> call UDF's flush method
if some operator requires final checkpoints
    snapshotState()
    notifyCheckpointComplete()
end if
close() --> call UDF's close method

Since currently the close() is only called in normal finish and dispose() will be called in both failover and normal case, for compatibility, I think we may
have to postpone the change to a single close() method to version 2.0 ? 

3. Regarding the name and position of flush() method, I also agree with that we will need a separate method to mark the termination of the whole stream for 
multiple-input streams. Would it be also ok if we have some modification to the current BoundedXXInput interfaces to 

interface BoundedInput {
    void endInput() // marks the end of the whole streams, as flush() does. 
}

@deprecated // In the future we could remove this interface
interface BoundedOneInput extends BoundedInput {}

interface BoundedMultiInput extends BoundedInput {
      void endInput(int i);

      default void endInput() {} // For compatibility 
}

If operator/UDF does not care about the end of a single input, then it could directly implement the BoundedInput interface. The possible 
benefit to me is that we might be able to keep only one concept for marking the end of stream, especially for the operators with only 
one input. 

Very thanks for all the deep insights and discussions!

Best,
Yun


------------------------------------------------------------------
From:Dawid Wysakowicz <dw...@apache.org>
Send Time:2021 Jun. 3 (Thu.) 21:21
To:dev <de...@flink.apache.org>; Till Rohrmann <tr...@apache.org>; Yun Gao <yu...@aliyun.com>
Cc:Piotr Nowojski <pn...@apache.org>; Guowei Ma <gu...@gmail.com>; Stephan Ewen <se...@apache.org>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi all,
Thanks for the very insightful discussion. I'd like to revive the effort of FLIP-147. First of all, from my side I'd like to say that I am really interested in helping that happen in the upcoming 1.14 release.
I agree with Till that the final checkpoints and global commits are mostly orthogonal. Similarly as Till, I'd suggest to first focus on the final checkpoints, while just keeping in mind we should not make assumptions that would make it impossible to implement the global commits. So far I do not see such risk from the discussion.
Going back to the final checkpoints issue. I think the only outstanding issue is which methods we want to use for flushing/closing both operators and UDFs just before performing the final checkpoint. As pointed out to me by Piotr, I am mentioning UDFs here as well, because we need a way for users using the Public API to benefit from the final checkpoint (bear in mind that e.g. TwoPhaseCommitSinkFunction which is implemented by our Kafka sink operates on the UDF level). Right now RichFunction has no method which could be called just before the final checkpoint that would say "flush" all intermediate state now and prepare for the final checkpoint. I'd suggest introducing an additional interface e.g. (name to be determined)
interface Flushable<T> {
   void flush(Collector<T> out)
}
Additionally we would need to introduce a similar method on the StreamOperator level. Currently we have two methods that are called at the end of operator lifecycle: 
close 
dispose 
The usage of the two methods is a bit confusing. Dispose is responsible for closing all open resources and is supposed to be called in case of a failure. On the other hand the close is a combination of a non-existent "flush" method we need and dispose for closing resources in case of a successful run. I'd suggest to clear it a bit. We would introduce a proper "flush" method which would be called in case of a successful finishing of an operator. Moreover we would make "close" deal only with closing any open resources, basically taking over the role of the dispose, which we would deprecate.
Lastly, I'd like to say why I think it is better introduce a new "flush" method instead of using the "endInput" method of BoundedOne/MultiInput. That method is called per input, which means each operator would need to keep track of which inputs were already closed internally and react differently if all of the inputs were closed. With an explicit "flush" method we do not have such a problem as the input bookkeeping happens on the StreamTask level.
Let me know what you think. I'd sync with Yun Gao and if there are no objections we will extend the FLIP page with necessary changes.
Best,
Dawid


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

Posted by Dawid Wysakowicz <dw...@apache.org>.
Hi all,

Thanks for the very insightful discussion. I'd like to revive the effort
of FLIP-147. First of all, from my side I'd like to say that I am really
interested in helping that happen in the upcoming 1.14 release.

I agree with Till that the final checkpoints and global commits are
mostly orthogonal. Similarly as Till, I'd suggest to first focus on the
final checkpoints, while just keeping in mind we should not make
assumptions that would make it impossible to implement the global
commits. So far I do not see such risk from the discussion.

Going back to the final checkpoints issue. I think the only outstanding
issue is which methods we want to use for flushing/closing both
operators and UDFs just before performing the final checkpoint. As
pointed out to me by Piotr, I am mentioning UDFs here as well, because
we need a way for users using the Public API to benefit from the final
checkpoint (bear in mind that e.g. TwoPhaseCommitSinkFunction which is
implemented by our Kafka sink operates on the UDF level). Right now
RichFunction has no method which could be called just before the final
checkpoint that would say "flush" all intermediate state now and prepare
for the final checkpoint. I'd suggest introducing an additional
interface e.g. (name to be determined)

|interface Flushable<T> {|

|   void flush(Collector<T> out)||
|

|}|

Additionally we would need to introduce a similar method on the
StreamOperator level. Currently we have two methods that are called at
the end of operator lifecycle:

  * close
  * dispose

The usage of the two methods is a bit confusing. Dispose is responsible
for closing all open resources and is supposed to be called in case of a
failure. On the other hand the close is a combination of a non-existent
"flush" method we need and dispose for closing resources in case of a
successful run. I'd suggest to clear it a bit. We would introduce a
proper "flush" method which would be called in case of a successful
finishing of an operator. Moreover we would make "close" deal only with
closing any open resources, basically taking over the role of the
dispose, which we would deprecate.

Lastly, I'd like to say why I think it is better introduce a new "flush"
method instead of using the "endInput" method of BoundedOne/MultiInput.
That method is called per input, which means each operator would need to
keep track of which inputs were already closed internally and react
differently if all of the inputs were closed. With an explicit "flush"
method we do not have such a problem as the input bookkeeping happens on
the StreamTask level.

Let me know what you think. I'd sync with Yun Gao and if there are no
objections we will extend the FLIP page with necessary changes.

Best,

Dawid

On 29/03/2021 18:30, Till Rohrmann wrote:
> Thanks a lot for all your input. To sum up the discussion so far:
>
> ## Final checkpoints
>
> We currently agree on favouring a single final checkpoint which can shut
> down the topology. In order to support this we need to be able to create a
> checkpoint after an operator has finished producing results.
>
> If we want to send checkpoint barriers through the topology this means that
> a task must not close the network connection when it sees "logical end of
> data". Instead, on the "logical end of data" the contained operator should
> flush all of its records. This means that we need to introduce a new event
> "logical end of data" and API calls to signal an operator that it should
> flush its data and that it should shut down.
>
> Given the available methods, `endInput` could be used for signalling the
> "logical end of data" and `dispose` for shutting the operator down. A task
> will only shut down and send an "EndOfPartitionEvent" which closes the TCP
> connection if all of its inputs have shut down and if it has completed a
> final checkpoint.
>
> ## Global commits
>
> Now a somewhat related but also orthogonal issue is how to support a global
> commit. A global commit is a commit where the external artefacts of a
> checkpoint become visible all at once. The global commit should be
> supported for streaming as well as batch executions (it is probably even
> more important for batch executions). In general, there could be different
> ways of implementing the global commit mechanism:
>
> 1. Collect global commit handles on the JM and run the global commit action
> on the JM
> 2. Collect global commit handles in a parallelism 1 operator which performs
> the global commit action
>
> Approach 2. would probably require to be able to send records from the
> snapshotState() method which would be the global commit handles. Both
> approaches would have to persist some kind of information in the checkpoint
> which allows redoing the global commit operation in case of a failover.
> Therefore, for approach 1. it would be required that we send the global
> commit handles to the JM from the snapshotState() method and not the
> notifyCheckpointComplete().
>
> A related question is in which order to execute the local and global commit
> actions:
>
> 1. Unspecified order
> 2. First local commits and then global commits
>
> Option 1. would be easier to implement and might already be good enough for
> most sinks.
>
> I would suggest treating final checkpoints and global commits as two
> related but separate things. I think it is fine to first try to solve the
> final checkpoints problem and then to tackle the global commits. This will
> help to decrease the scope of each individual feature.
>
> Cheers,
> Till
>
> On Fri, Mar 5, 2021 at 5:12 AM Yun Gao <yu...@aliyun.com> wrote:
>
>> Hi Piotr,
>>
>> Very thanks for the suggestions and thoughts!
>>
>>> Is this a problem? Pipeline would be empty, so EndOfPartitionEvent would
>> be traveling very quickly.
>>
>> No, this is not a problem, sorry I have some wrong thoughts here,
>> initially in fact I'm thinking on this issue raised by
>> @kezhu:
>>
>>> Besides this, will FLIP-147 eventually need some ways to decide whether
>> an operator need final checkpoint
>> @Yun @Guowei ?  @Arvid mentions this in earlier mail.
>>
>> For this issue itself, I'm still lean towards we might still need it, for
>> example, suppose we have a job that
>> do not need to commit anything on finished, then it do not need to wait
>> for checkpoint at all for normal
>> finish case.
>>
>>> Yes, but aren't we doing it right now anyway?
>> `StreamSource#advanceToEndOfEventTime`?
>>
>> Yes, we indeed have advancedEndOfEventTime for both legacy and new
>> sources, sorry for the overlook.
>>
>>> Is this the plan? That upon recovery we are restarting all operators,
>> even those that have already finished?
>> Certainly it's one of the possibilities.
>>
>> For the first version we would tend to use this way since it is easier to
>> implement, and we should always need
>> to consider the case that tasks are started but operators are finished
>> since there might be also tasks with part
>> of operators finished. For the long run I think we could continue to
>> optimize it via not restart the finished tasks
>> at all.
>>
>>> Keep in mind that those are two separate things, as I mentioned in a
>> previous e-mail:
>>>> II. When should the `GlobalCommitHandle` be created? Should it be
>> returned from `snapshotState()`, `notifyCheckpointComplete()` or somewhere
>> else?
>>>> III. What should be the ordering guarantee between global commit and
>> local commit, if any? Actually the easiest to implement would be undefined,
>> but de facto global commit happening before local commits (first invoke > `notifyCheckpointComplete()`
>> on the `OperatorCoordinator` and either after or in parallel send
>> `notifyCheckpointComplete()` RPCs). As far as I can tell, undefined order
>> should work for the use cases that I'm aware of.
>>> We could create the `GlobalCommitHandle` in
>> `StreamOperator#snapshotState()`, while we could also ensure that
>> `notifyCheckpointComplete()` is called on the `OperatorCoordinator` AFTER
>> all of the operators have successfully > processed
>> `notifyCheckpointComplete()`. This would be more difficult to implement,
>> hence I would prefer "undefined" behaviour here, but it's probably possible.
>>
>> Very thanks for the further explanation, and I also totally agree with
>> that the two are separate and we could think on them
>> distinctly. Regarding the order, I would still tend to we support the
>> ordered case, since the sinks' implementation seem to depend
>> on this functionality.
>>
>> Best,
>>  Yun
>>
>> ------------------------------------------------------------------
>> From:Piotr Nowojski <pn...@apache.org>
>> Send Time:2021 Mar. 4 (Thu.) 22:56
>> To:Kezhu Wang <ke...@gmail.com>
>> Cc:Till Rohrmann <tr...@apache.org>; Guowei Ma <gu...@gmail.com>;
>> dev <de...@flink.apache.org>; Yun Gao <yu...@aliyun.com>;
>> jingsonglee0@gmail.com <ji...@gmail.com>
>> Subject:Re: Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
>> Finished
>>
>> Hi Yun and Kezhu,
>>
>>> 1. We might introduce a new type of event to notify the endOfInput()
>> though the graph first, and then A/B waits for the final
>> checkpoint, then A emit EndOfPartitionEvent to exit all the tasks as now.
>>
>> As I mentioned in one of the PRs, I would opt for this solution.
>>
>>>  if we go towards 1, the tasks would still need to exit from the source,
>> and if we go towards 2/3, we could be able to allow these
>> tasks to finish first.
>>
>> Is this a problem? Pipeline would be empty, so EndOfPartitionEvent would
>> be traveling very quickly.
>>
>>> should we also need to do it for normal exit
>> Yes, but aren't we doing it right now anyway?
>> `StreamSource#advanceToEndOfEventTime`?
>>
>>> If so, since now for recovery after some tasks finished we would first
>> start all the tasks and stop the finished tasks directly
>>
>> Is this the plan? That upon recovery we are restarting all operators, even
>> those that have already finished? Certainly it's one of the possibilities.
>>
>>> For example, the global committer handler might be write meta store for
>> FileSystem/Hive sinks, and these should happen after all the pending
>>> files are renamed to the final ones, otherwise the downstream jobs might
>> miss some files if they relies on the meta store to identify ready
>> partitions.
>>> Thus we would have to emit the global-committer-handler after
>> notifyCheckpointComplete. But since we could be able to know the list of
>> files
>>> to rename in snapshotState(), we could create the
>> global-committer-handler and store them there.
>>
>> Keep in mind that those are two separate things, as I mentioned in a
>> previous e-mail:
>>> II. When should the `GlobalCommitHandle` be created? Should it be
>> returned from `snapshotState()`, `notifyCheckpointComplete()` or somewhere
>> else?
>>> III. What should be the ordering guarantee between global commit and
>> local commit, if any? Actually the easiest to implement would be undefined,
>> but de facto global commit happening before local commits (first invoke
>> `notifyCheckpointComplete()` on the `OperatorCoordinator` and either after
>> or in parallel send `notifyCheckpointComplete()` RPCs). As far as I can
>> tell, undefined order should work for the use cases that I'm aware of.
>>
>> We could create the `GlobalCommitHandle` in
>> `StreamOperator#snapshotState()`, while we could also ensure that
>> `notifyCheckpointComplete()` is called on the `OperatorCoordinator` AFTER
>> all of the operators have successfully processed
>> `notifyCheckpointComplete()`. This would be more difficult to implement,
>> hence I would prefer "undefined" behaviour here, but it's probably possible.
>>
>> Kezhu:
>>
>>> Is `endOfInput` in your proposal same as `BoundedOneInput.endInput` ?
>> Yes it's the same. Sorry for a typo, somehow I was convinced its
>> `endOfInput` not simple `endInput` :)
>>
>> Piotrek
>>
>> czw., 4 mar 2021 o 11:09 Kezhu Wang <ke...@gmail.com> napisał(a):
>>
>>
>>

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi all,

Very thanks for all the discussions!

Regarding the operator API on exit, I'm also glad that we are reaching consistency roughly. Based on the previous discussions
I'm also support doing the final checkpoint after the previous "close" method, and we rename these methods to make them
more clear. But I'm still have concerns on how we achieve the final checkpoint after emitting the EndOfPartitionEvent, which 
has closed the network channels ? Suppose we have A -> B and both A and B need to wait for the final checkpoint, there might
be several options from my side:
1. We might introduce a new type of event to notify the endOfInput() though the graph first, and then A/B waits for the final 
checkpoint, then A emit EndOfPartitionEvent to exit all the tasks as now.
2. We directly use EndOfPartitionEvent to notify the endOfInput() throw the graph first, but we also notify JM (or at least 
CheckpointCoordinator) that these tasks are in a new state TERMINATING (or FINISHING). Then when triggering checkpoint
the CheckpointCoordinator would treat these tasks differently by not considering the edges emitted EndOfPartitionEvent. In this
case, we would trigger A and B separately and there is not need for A to broadcast the barriers to B.
3. We still use EndOfPartitionEvent  to notify the endOfInput() throw the graph first and do not notify JM. Then for all the checkpoints 
we would need to trigger all the running tasks. Like in this case, both A and B are running and CheckpointCoordinator do not know 
whether the network between them are closed, then it has to assume the worst case and trigger both A and B. But this option would
introduce more overhead compared with that we only trigger sources and broadcast barriers in task side. 

For whether we allow the operators and tasks that do not need to commit side effect to exit first, I think it is also related to the above
options: if we go towards 1, the tasks would still need to exit from the source, and if we go towards 2/3, we could be able to allow these
tasks to finish first.

Regarding the watermark,
> There is a pre-existing issue
> that watermarks are not checkpointed/stored on state, and there was/is now
> clear answer how we should handle this as far as I remember. One
> problematic case are two/multiple input tasks or UnionInputGate, where
> combined watermark is the min of all inputs (held in memory). The problem
> so far is a bit benign, as after recovery we are losing the combined
> watermark value, but it's being slowly/lazily restored, as new watermarks
> are sent from the sources. With finished sources that won't be a case.

Very thanks for the further explain and it should indeed be a problem. Since now for stop-with-savepoint --drain we always emit
advance the watermark to MAX, should we also need to do it for normal exit ? If so, since now for recovery after some tasks finished
we would first start all the tasks and stop the finished tasks directly, I think for simplicity we could first emit a new MAX watermark 
from the sources before or with EndOfpartitionEvent as till suggested, and for the long run we could also consider snapshotting the
min watermark if we are going to not start the finished tasks directly.

Regarding the global-commit-handles,
I also like to proposal of the global-committer-handler, From the sinks' view I'm also lean towards emit these handler after notifyCheckpointComplete, 
but we could create these handlers in snapshotState() so that we could also include them in the checkpoint.

 For example, the global committer handler might be write meta store for FileSystem/Hive sinks, and these should happen after all the pending
files are renamed to the final ones, otherwise the downstream jobs might miss some files if they relies on the meta store to identify ready partitions.
Thus we would have to emit the global-committer-handler after notifyCheckpointComplete. But since we could be able to know the list of files
to rename in snapshotState(), we could create the global-committer-handler and store them there. 

Also since we might want to keep the order, the operator coordinator might not relies on its own notifyCheckpointComplete() notification,
but wait for the the operator to notify it about the checkpoint complete after the operator has finished its processing first.

Best,
Yun



------------------------------------------------------------------
From:Piotr Nowojski <pn...@apache.org>
Send Time:2021 Mar. 4 (Thu.) 17:16
To:Kezhu Wang <ke...@gmail.com>
Cc:dev <de...@flink.apache.org>; Yun Gao <yu...@aliyun.com>; jingsonglee0@gmail.com <ji...@gmail.com>; Guowei Ma <gu...@gmail.com>; Till Rohrmann <tr...@apache.org>
Subject:Re: Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi Kezhu,

What do you mean by “end-flushing”? I was suggesting to just keep `endOfInput()` and `dispose()`. Are you suggesting to have a one `endFlushing()` method, that is called after quiescing timers/mailbox, but before final checkpoint and `dispose()`? Are we sure we really need this extra call? Note. If we don't need it at the moment, we could always introduce it in the future, while if we don't and won't need it, why complicate the API?

About the idea of returning the global-commit-handle from `notifyCheckpointComplete()` call. Note it will be more difficult to implement, as `CheckpointCoordinator` will need to have one extra stage of waiting for some actions to complete. Implementation will probably be easier if we return the global-commit-handle from `snapshotState()` call.

Also, `global-commit-handles` will need to be part of the checkpoint. They will need to be restored/recovered in case of failure. Because of that it might be actually impossible to implement those handles as returned from `notifyCheckpointComplete()`. In this solution we would be in a precarious position if the main checkpoint succeeded, CheckpointCoordinator would start issuing `notifyCheckpointComplete()`, but persisting of the handles would fail/keep failing. How would we recover from such a situation? We can not recover to a previous checkpoint (`notifyCheckpointComplete()` were already issued), but at the same time the current checkpoint is not fully completed (global-commit-handles can not be checkpointed).

Best,
Piotrek



czw., 4 mar 2021 o 06:33 Kezhu Wang <ke...@gmail.com> napisał(a):



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

Posted by Piotr Nowojski <pn...@apache.org>.
Hi Yun and Kezhu,

> 1. We might introduce a new type of event to notify the endOfInput()
though the graph first, and then A/B waits for the final
checkpoint, then A emit EndOfPartitionEvent to exit all the tasks as now.

As I mentioned in one of the PRs, I would opt for this solution.

>  if we go towards 1, the tasks would still need to exit from the source,
and if we go towards 2/3, we could be able to allow these
tasks to finish first.

Is this a problem? Pipeline would be empty, so EndOfPartitionEvent would be
traveling very quickly.

> should we also need to do it for normal exit

Yes, but aren't we doing it right now anyway?
`StreamSource#advanceToEndOfEventTime`?

> If so, since now for recovery after some tasks finished we would first
start all the tasks and stop the finished tasks directly

Is this the plan? That upon recovery we are restarting all operators, even
those that have already finished? Certainly it's one of the possibilities.

> For example, the global committer handler might be write meta store for
FileSystem/Hive sinks, and these should happen after all the pending
> files are renamed to the final ones, otherwise the downstream jobs might
miss some files if they relies on the meta store to identify ready
partitions.
> Thus we would have to emit the global-committer-handler after
notifyCheckpointComplete. But since we could be able to know the list of
files
> to rename in snapshotState(), we could create the
global-committer-handler and store them there.

Keep in mind that those are two separate things, as I mentioned in a
previous e-mail:
> II. When should the `GlobalCommitHandle` be created? Should it be
returned from `snapshotState()`, `notifyCheckpointComplete()` or somewhere
else?
> III. What should be the ordering guarantee between global commit and
local commit, if any? Actually the easiest to implement would be undefined,
but de facto global commit happening before local commits (first invoke
`notifyCheckpointComplete()` on the `OperatorCoordinator` and either after
or in parallel send `notifyCheckpointComplete()` RPCs). As far as I can
tell, undefined order should work for the use cases that I'm aware of.

We could create the `GlobalCommitHandle` in
`StreamOperator#snapshotState()`, while we could also ensure that
`notifyCheckpointComplete()` is called on the `OperatorCoordinator` AFTER
all of the operators have successfully processed
`notifyCheckpointComplete()`. This would be more difficult to implement,
hence I would prefer "undefined" behaviour here, but it's probably possible.

Kezhu:

> Is `endOfInput` in your proposal same as `BoundedOneInput.endInput` ?

Yes it's the same. Sorry for a typo, somehow I was convinced its
`endOfInput` not simple `endInput` :)

Piotrek

czw., 4 mar 2021 o 11:09 Kezhu Wang <ke...@gmail.com> napisał(a):

> Hi Piotrek,
>
> For “end-flushing”, I presented no new candidates. I uses “end-flushing”
> to avoid naming issues. It should be “close” in allowing checkpoint after
> “close” while “endOfInput” in your approach. What I tried to express is we
> can do some backporting work to mitigate possible breaking changes in your
> approach.
>
> To confirm to myself that whether I understand your approach correctly, I
> want to confirm somethings to avoid misleading from my side:
> * Is `endOfInput` in your proposal same as `BoundedOneInput.endInput` ?
>
> I (also ?) think there are overlaps between `BoundedOneInput.endInput` and
> `close`. If yes, I like your proposal to squash them to one and deprecate
> `BoundedOneInput.endInput`.
>
> > Also, `global-commit-handles` will need to be part of the checkpoint.
>
> I had to admit that I overlooked this, sorry for this. In my previous
> suggestion, I assumed `global-commit-handles` is not part of checkpoint.
> Checkpoint will complete and persist as before. The changed parts are after
> checkpoint considered completed.
>
>
> Best,
> Kezhu Wang
>
>
> On March 4, 2021 at 17:15:53, Piotr Nowojski (pnowojski@apache.org) wrote:
>
> Hi Kezhu,
>
> What do you mean by “end-flushing”? I was suggesting to just keep
> `endOfInput()` and `dispose()`. Are you suggesting to have a one
> `endFlushing()` method, that is called after quiescing timers/mailbox, but
> before final checkpoint and `dispose()`? Are we sure we really need this
> extra call? Note. If we don't need it at the moment, we could always
> introduce it in the future, while if we don't and won't need it, why
> complicate the API?
>
> About the idea of returning the global-commit-handle from
> `notifyCheckpointComplete()` call. Note it will be more difficult to
> implement, as `CheckpointCoordinator` will need to have one extra stage of
> waiting for some actions to complete. Implementation will probably be
> easier if we return the global-commit-handle from `snapshotState()` call.
>
> Also, `global-commit-handles` will need to be part of the checkpoint. They
> will need to be restored/recovered in case of failure. Because of that it
> might be actually impossible to implement those handles as returned from
> `notifyCheckpointComplete()`. In this solution we would be in a precarious
> position if the main checkpoint succeeded, CheckpointCoordinator would
> start issuing `notifyCheckpointComplete()`, but persisting of the handles
> would fail/keep failing. How would we recover from such a situation? We can
> not recover to a previous checkpoint (`notifyCheckpointComplete()` were
> already issued), but at the same time the current checkpoint is not fully
> completed (global-commit-handles can not be checkpointed).
>
> Best,
> Piotrek
>
>
>
> czw., 4 mar 2021 o 06:33 Kezhu Wang <ke...@gmail.com> napisał(a):
>
>> Hi all,
>>
>> Glad to see convergence here and FLINK-21133:
>> 1. We all prefer single final checkpoint for task not individual
>> checkpoints for each operators.
>> 2. To above goal, if we have to breaking something, we will.
>> 3. Don’t allow recording emitting in `notifyCheckpointComplete`.
>>
>> For “end-flushing”, I think both approaches should function in reality,
>> but we also have options/responsibilities to mitigate effect of breaking
>> changes:
>> A. Allowing checkpoint after “close”. Introduce config option to forbid
>> this during migrating releases.
>> B. Renaming “close” to “other-end-flushing-method”. We can backport that
>> newly introducing “end-flushing”(as empty default method) to earlier
>> releases in following patch releases. The backporting “end-flushing” will
>> be called just before “close” in future patch releases. We could call
>> “close” just before “dispose" in future releases and `final` it in
>> `AbstractStreamOperator` when ready(to break user side code).
>>
>> If breaking change for this “end-flushing” in inevitable, I am kind of
>> prefer renaming and backport approach. It is a chance for us to rethink the
>> whole thing and discard misleading “close”(currently it is mixed/misused
>> with “end-flushing” and “cleanup-resource” though javadoc claims only
>> “end-flushing, this could also be considered as a bug though).
>>
>> Besides this, will FLIP-147 eventually need some ways to decide whether
>> an operator need final checkpoint @Yun @Guowei ?  @Arvid mentions this in
>> earlier mail.
>>
>>
>> For the two phase commit, @Piotrek I like your idea. I think that
>> “commit-handle” could be return to checkpoint-coordinator through
>> `notifyCheckpointComplete`. This way that “commit-handle” might be reused
>> by operator-coordinator’s `notifyCheckpointComplete`. Suppose following
>> changes:
>>
>> 1. `CompletableFuture<SerializedValue<?>>
>> notifyCheckpointCompleteAsync()` in operator.
>> 2. `CompletableFuture<Void> notifyCheckpointCompleteAsync(Map<Integer,
>> CompletableFuture<SerializedValue>> subtasks)` in operator coordinator.
>>
>> These changes need support from:
>> * Checkpoint coordinator level to bridge operator and coordinator through
>> task
>> * Operator level to compat existing `notifyCheckpointComplete`
>>
>> The checkpoint procedure will looks like:
>> 1. Trigger checkpoint for operator coordinator.
>> 2. If above succeeds, trigger tasks checkpoint. Abort otherwise.
>> 3. If all above succeeds, complete current checkpoint. Abort otherwise.
>> 4. If job fails after, restore from above “completed” checkpoint.
>> 5. Notify checkpoint completion to tasks.
>> 6. Notify checkpoint completion to coordinators.
>> 7. Wait step#5 and step#6 to succeed. Now it is real completed. Either
>> this succeed or job failed in meantime ? May be other concurrent conditions.
>>
>> With these changes, migration FLIP-143 sink to operator coordinator
>> should be easy.
>>
>> It will definitely complicate currently complex checkpoint coordinator as
>> @Till mentioned in FLINK-21133.
>>
>>
>> Best,
>> Kezhu Wang
>>
>> On March 3, 2021 at 01:09:50, Piotr Nowojski (pnowojski@apache.org)
>> wrote:
>>
>> Hi,
>>
>> Thanks for reminding me. I think FLIP-147 will have to deal in one way or
>> another with the (re?)emitting MAX_WATERMARK. There is a pre-existing
>> issue
>> that watermarks are not checkpointed/stored on state, and there was/is
>> now
>> clear answer how we should handle this as far as I remember. One
>> problematic case are two/multiple input tasks or UnionInputGate, where
>> combined watermark is the min of all inputs (held in memory). The problem
>> so far is a bit benign, as after recovery we are losing the combined
>> watermark value, but it's being slowly/lazily restored, as new watermarks
>> are sent from the sources. With finished sources that won't be a case.
>>
>> I've spent more time thinking about the two phase commit issue, and I
>> agree
>> it would be best to have a single final checkpoint for the operators. The
>> more I think about it, the more I'm leaning toward the
>> "OperatorCoordinator" approach. Yes, it would need the extra complexity
>> of
>> running user code on the JobManager, but that's already at least
>> partially
>> done (FLIP-27). On the other hand it would allow us to keep the runtime
>> code cleaner and simpler I think. It's still an invasive change.
>>
>> 1. We forbid emitting records from
>> `notifyCheckpointComplete`/`snapshotState` calls, or at the very least
>> from
>> the final calls.
>> 2. We need to clarify/clean up contracts for both operators AND user
>> functions when it comes to `endOfInput()`, `close()` and `dispose()`
>> methods. For example:
>> a) we keep `endOfInput()` on the operators level and add an equivalent of
>> it to the user functions
>> b) we move the "flushing buffered records" part of the `close()` contract
>> to `endOfInput()`
>> c) we remove `close()` methods altogether (IMO it's better to have an
>> explicit false conflict, rather than a silent real one)
>> d) we keep `dispose()` on the operator level for release resources, and
>> we add such method to the user functions
>> 3. We change (or add another version) a return type of either `void
>> snapshotState(...)` or `void notifyCheckpointComplete(...)` calls, so
>> that
>> they will return some kind of `GlobalCommitHandle`.
>>
>> Especially this point 2. is intrusive. Now, putting all of that together.
>> When a task receives a logical "end of input" signal (current
>> `EndOfPartitionEvent`), it begins shutting down procedure (let's assume
>> we
>> create `GlobalCommitHandle` in `snapshotState`).
>>
>> 0. n = 1
>> 1. endOfInput on the n'th operator
>> 2. quisec timers and mailbox executor for n'th operator
>> // after this point n'th operator is forbidden to emit any new records
>> 3. n += 1, and go to 1., until we handle all of the operators
>> 4. wait for final checkpoint, issue `snapshotState` and collect
>> `GlobalCommitHandle`s and send them to the JM
>> 5. JM/CheckpointCoordinator collects all `GlobalCommitHandles`. They are
>> an
>> integral part of the checkpoint. They could be added to the respective
>> `OperatorCoordinator`s via `handleEventFromOperator` as an
>> `OperatorEvent`.
>> 6. When checkpoint is considered completed, `notifyCheckpointComplete`
>> are
>> being issued, both on the operators, and `OperatorCoordinator` - local
>> commit and global commits would be happening at the same time, or even
>> global commit code would be executed before local final commit in this
>> version.
>> 7. Only now can we finally dispose of all operators in the task.
>>
>> If there is a failure, especially before we manage to perform all
>> `notifyCheckpointComplete` calls (for example on the
>> `OperatorCoordinator`), we would need to recover from last "completed"
>> checkpoint, recover attached `GlobalCommitHandle`s , and re-commit them.
>> It
>> also means we need to recover already finished operators, that will never
>> process any records, just to issue the final `notifyCheckpointComplete`
>> call and make sure they commit their external side effects.
>>
>> There are at least a couple of questions about this general idea:
>> I. Naming of the methods (I suggested to drop `close()` and keep in the
>> operators and introduce in the functions `endOfInput()` and `dispose()`
>> II. When should the `GlobalCommitHandle` be created? Should it be
>> returned
>> from `snapshotState()`, `notifyCheckpointComplete()` or somewhere else?
>> III. What should be the ordering guarantee between global commit and
>> local
>> commit, if any? Actually the easiest to implement would be undefined, but
>> de facto global commit happening before local commits (first invoke
>> `notifyCheckpointComplete()` on the `OperatorCoordinator` and either
>> after
>> or in parallel send `notifyCheckpointComplete()` RPCs). As far as I can
>> tell, undefined order should work for the use cases that I'm aware of.
>> IV. Can we reuse `OperatorCoordinator` abstraction for this purpose?
>> Currently it's being used to induce checkpoints with FLIP-27 sources as
>> far
>> as I understand, here we would be using it for different purposes.
>>
>> Best,
>> Piotrek
>>
>> pon., 1 mar 2021 o 07:30 Yun Gao <yu...@aliyun.com> napisał(a):
>>
>> > (Sorry that I repeat this mail since the last one is not added into the
>> > same mail list thread,
>> > very sorry for the inconvenience)
>> >
>> > Hi all,
>> >
>> > Very thanks for all the deep thoughts!
>> >
>> > > How to implement the stop-with-savepoint --drain/terminate command
>> with
>> > > this model: One idea could be to tell the sources that they should
>> stop
>> > > reading. This should trigger the EndOfPartitionEvent to be sent
>> > > downstream.
>> > > This will transition all operators into the TERMINATING state.
>> > >
>> > > Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
>> > > returned. To achieve above, possible works should be required:
>> > > * Promote `EndOfPartitionEvent` from `Task` to `StreamTask`. This may
>> > > have some interferences with BatchTask or network io stack.
>> > > * Or introducing stream task level `EndOfUserRecordsEvent`(from
>> PR#14831
>> > > @Yun @Piotr)
>> > > * Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.
>> >
>> > I also have similar concern with Kezhu for the issue whether do we need
>> to
>> > introduce a new message
>> > to notify the operators to endOfInput/close ? The main concerns of
>> reusing
>> > that EndOfPartitionEvent
>> > is that
>> > 1. The EndOfPartitionEvent is currently emitted in Task instead of
>> > StreamTask, we would need some
>> > refactors here.
>> > 2. Currently the InputGate/InputChannel would be released after the
>> > downstream tasks have received
>> > EndOfPartitionEvent from all the input channels, this would makes the
>> > following checkpoint unable to
>> > perform since we could not emit barriers to downstream tasks ?
>> >
>> > Regarding the MAX_WATERMARK, I still not fully understand the issue
>> since
>> > it seems to me
>> > that now Flink won't snapshot the watermark now? If the job failover,
>> the
>> > window operator
>> > would reload all the pending windows before flushed by MAX_WATERMARK
>> and
>> > when the
>> > job finish again, it would re-emit the MAX_WATERMARK?
>> >
>> > Best,
>> > Yun
>> >
>> >
>> > ------------------------------------------------------------------
>> > From:Kezhu Wang <ke...@gmail.com>
>> > Send Time:2021 Mar. 1 (Mon.) 01:26
>> > To:Till Rohrmann <tr...@apache.org>
>> > Cc:Piotr Nowojski <pi...@gmail.com>; Guowei Ma <
>> > guowei.mgw@gmail.com>; dev <de...@flink.apache.org>; Yun Gao <
>> > yungao.gy@aliyun.com>; jingsonglee0@gmail.com <ji...@gmail.com>
>> > Subject:Re: Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
>> > Finished
>> >
>> > In “stop-with-savepoint —drain”, MAX_WATERMARK is not an issue. For
>> normal
>> > finishing task, not allowing unaligned checkpoint does not solve the
>> > problem as MAX_WATERMARK could be persisted in downstream task. When
>> > scenario @Piotr depicted occurs, downstream(or further downstream)
>> window
>> > operator will count all inputs as late.
>> >
>> > > If we ensure that the MAX_WATERMARK is only persisted in state if a
>> > recovery will trigger immediately the shut down of this operator, then
>> it
>> > shouldn't be an issue.
>> >
>> > You are right in case the assumption holds, I have same thought as you
>> > before. But I am kind of worry about whether it is too prefect to be
>> > fragile. This requires strong guarantee from implementation that a
>> recovery
>> > from TERMINATING stage should go directly to that stage.
>> >
>> > > I think the MAX_WATERMARK event should be sent either just before or
>> with
>> > the EndOfPartitionEvent.
>> >
>> > I prefer “with the EndOfPartitionEvent”. EndOfPartitionEvent itself
>> already
>> > carry what ending MAX_WATERMARK try to express. May be we can reuse it
>> ?
>> > @Piotr
>> >
>> > A preview work[1] from @Yun in PR#14831 explains EndOfPartitionEvent as
>> > checkpoint barrier if there are pending checkpoints.
>> >
>> >
>> > [1]:
>> >
>> >
>> https://github.com/gaoyunhaii/flink/commit/1aaa80fb0afad13a314b06783c61c01b9414f91b#diff-d4568b34060bc589cd1354bd125c35aca993dd0c9fe9e4460dfed73501116459R177
>> >
>> >
>> > Best,
>> > Kezhu Wang
>> >
>> > On February 28, 2021 at 21:23:31, Till Rohrmann (trohrmann@apache.org)
>> > wrote:
>> >
>> > I think you are right with the problem of endOfInput. endOfInput should
>> not
>> > be used to commit final results. In fact if this termination fails then
>> we
>> > might end up in a different outcome of the job which is equally valid
>> as
>> > the one before the failure.
>> >
>> > Concerning unaligned checkpoints, I think they don't play well together
>> > with draining a streaming pipeline. The problem is that in the draining
>> > case you want to process all records which are still in flight but
>> > unaligned checkpoints don't guarantee this as they can jump in flight
>> > records.
>> >
>> > I think the MAX_WATERMARK event should be sent either just before or
>> with
>> > the EndOfPartitionEvent. If we ensure that the MAX_WATERMARK is only
>> > persisted in state if a recovery will trigger immediately the shut down
>> of
>> > this operator, then it shouldn't be an issue.
>> >
>> > Cheers,
>> > Till
>> >
>> > On Sun, Feb 28, 2021 at 9:06 AM Kezhu Wang <ke...@gmail.com> wrote:
>> >
>> > > Hi Till,
>> > >
>> > > Just for bookkeeping, some observations from current implementation.
>> > >
>> > > > With this model, the final checkpoint is quite simple because it is
>> > > ingrained in the lifecycle of an operator. Differently said an
>> operator
>> > > will only terminate after it has committed its side effects and seen
>> the
>> > > notifyCheckpointComplete message (if it is stateful).
>> > >
>> >
>> > > Currently, we could not mark this operator(or subtask) as terminated
>> since
>> > > result of `notifyCheckpointComplete`(possible side effect committing)
>> is
>> > > not taken into account of the belonging checkpoint. The job has to
>> run to
>> > > next safe point(finished or next checkpoint success) to be marked as
>> > > “terminated”.
>> > >
>> > > > How to implement the stop-with-savepoint --drain/terminate command
>> with
>> > > this model: One idea could be to tell the sources that they should
>> stop
>> > > reading. This should trigger the EndOfPartitionEvent to be sent
>> > > downstream.
>> > > This will transition all operators into the TERMINATING state.
>> > >
>> > > Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
>> > > returned. To achieve above, possible works should be required:
>> > > * Promote `EndOfPartitionEvent` from `Task` to `StreamTask`. This may
>> > > have some interferences with BatchTask or network io stack.
>> > > * Or introducing stream task level `EndOfUserRecordsEvent`(from
>> PR#14831
>> > > @Yun @Piotr)
>> > > * Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.
>> > >
>> > > Besides this, I would like to quote some discussion from FLINK-21467
>> > > between @Piotr and me:
>> > >
>> > > From @Piotr
>> > > > Note, that it's not only that endOfInput can be called multiple
>> times.
>> >
>> > > There is a very remote possibility that the following scenario will
>> happen:
>> > > 1. checkpoint is taken (successfully)
>> > > 2. sources are finishing
>> > > 3. endOfInput is issued
>> > > 4. job fails
>> > > 5. job restarts to checkpoint 1.
>> > > 6. after failover, because of some non deterministic logic in the
>> source,
>> > > sources are not finishing
>> > >
>> > > From me
>> > > > But I think there is little work Flink can do to cope with this
>> kind of
>> > > issues. The checkpoint could be a savepoint triggered from user side
>> and
>> > > the "non deterministic logic" could be a change from user(eg.
>> changing of
>> > > stoppingOffsets in KafkaSource).
>> > >
>> >
>> > > > I think the "non deterministic logic" could cause trouble in
>> combination
>> > > with unaligned checkpoint and downstream window operator. Unaligned
>> > > checkpoint will persist "MAX_WATERMARK" in state, after restarting,
>> > > "MAX_WATERMARK" will cause downstream window operator ignores all
>> future
>> > > inputs.
>> > >
>> > > FLIP-147 demands no new records from end-of-stream-flushing, but
>> source
>> > > will emit “MAX_WATERMARK” on ending. Previously, I thought it is not
>> a
>> >
>> > > valid issue, but turn out that it could cause trouble under scenario
>> listed
>> > > by @Piotr if I am not wrong.
>> > >
>> > >
>> > > PR#14831: https://github.com/apache/flink/pull/14831
>> > > FLINK-21467: https://issues.apache.org/jira/browse/FLINK-21467
>> > >
>> > >
>> > > Best,
>> > > Kezhu Wang
>> > >
>> > > On February 27, 2021 at 18:12:20, Till Rohrmann (trohrmann@apache.org)
>>
>> > > wrote:
>> > >
>> > > Thanks for all your thoughts. I think we should further think through
>> >
>> > > whether to allow checkpoints after an operator has emitted all its
>> records
>> > > (e.g. after close is called currently) or not. I think by doing this
>> we
>> > > would nicely decouple the checkpoint taking from the operator
>> lifecycle
>> > > and
>> > > wouldn't need special checkpoints/savepoints for the final checkpoint
>> and
>> > > stop-with-savepoint --drain. Let me try to explain this a bit more
>> > > detailed.
>> > >
>> > > If we say an operator has the rough lifecycle RUNNING => TERMINATING
>> =>
>> > > TERMINATED where we go from RUNNING into TERMINATING after we have
>> seen
>> > > the
>> > > EndOfPartitionEvent and flushed all our records. The operator goes
>> from
>> > > TERMINATING => TERMINATED if it has persisted all its possible side
>> > > effects. Throughout all states, it is possible to trigger a
>> checkpoint. A
>> > > stateless operator will immediately go from TERMINATING to TERMINATED
>> > > whereas a stateful operator would wait for another checkpoint to be
>> > > triggered and successfully completed (notifyCheckpointComplete).
>> > >
>> > > With this model, the final checkpoint is quite simple because it is
>> > > ingrained in the lifecycle of an operator. Differently said an
>> operator
>> > > will only terminate after it has committed its side effects and seen
>> the
>> >
>> > > notifyCheckpointComplete message (if it is stateful). Here it is
>> important
>> > > to note that in the streaming case, different bounded operators can
>> > > terminate at different times. They don't have to terminate all with
>> the
>> > > same checkpoint.
>> > >
>> > > How to implement the stop-with-savepoint --drain/terminate command
>> with
>> > > this model: One idea could be to tell the sources that they should
>> stop
>> > > reading. This should trigger the EndOfPartitionEvent to be sent
>> > > downstream.
>> >
>> > > This will transition all operators into the TERMINATING state. Next
>> the JM
>> > > can trigger a checkpoint to shut the operators down and not to wait
>> for
>> > > the
>> > > next automatic checkpoint trigger event.
>> > >
>> > > By allowing checkpoints throughout the entire lifecycle of an
>> operator we
>> > > disallow sending records from notifyCheckpointComplete because this
>> > > message
>> > > will also be sent in the state TERMINATING where an operator has
>> already
>> > > produced all of its records.
>> > >
>> > > What do you think? Maybe this model overlooks some important cases.
>> One
>> > > downside is that we will break the operator API with changing the
>> > > lifecycle
>> > > of an operator.
>> > >
>> > > Cheers,
>> > > Till
>> > >
>> > >
>> > >
>> > > On Fri, Feb 26, 2021 at 7:29 AM Kezhu Wang <ke...@gmail.com> wrote:
>> > >
>> > > > Hi Yun,
>> > > >
>> > > > The termination phase I drafted depends on a de-fact that
>> >
>> > > > `notifyCheckpointComplete` is ignored currently after `close`.
>> Thus, any
>> > > > external materialization inside `close`(including `endInput`) or
>> > > > `notifyCheckpointComplete` is either not guaranteed or not
>> committed.
>> > > >
>> > > > I tried to emphasize no-breaking changes and post-pone migration in
>> > > later
>> > > > releases for this termination phase. But now, I am kind of worry
>> about
>> > > > whether it will cause long-term maintenance hell.
>> > > >
>> > > > Personally, I think allowing checkpoint after `close`(@Till
>> proposed
>> > > this
>> > > > in FLINK-21133) could be the minimal change. But there are
>> concerns:
>> > > > * It will break some existing code possibly in a silent way.(@Piotr
>> > > pointed
>> > > > this already in FLINK-21133)
>> > > > * I think the name `close` is kind of misleading. (@Piotr suggested
>> > > > renaming this to `finish` to not break code silently in
>> FLINK-21133)
>> > > >
>> > > > > stop-with-savepoint --drain would wait for a specific savepoint
>> > > >
>> > > > For stop-with-savepoint, the checkpoint barrier is already created
>> or
>> > > > received there.
>> > > >
>> > > >
>> > > > Best,
>> > > > Kezhu Wang
>> > > >
>> > > > On February 25, 2021 at 21:02:30, Yun Gao (yungao.gy@aliyun.com
>> > ) wrote:
>> > > >
>> > > > Hi all,
>> > > >
>> > > > Very thanks for the discussions!
>> > > >
>> > > >
>> > > >
>> > > > A. Regarding how to avoid emitting records in
>> notifyCheckpointComplete:
>> > > >
>> > > > Currently the structure of a new sink is writer -> committer ->
>> global
>> > > > committer and the paralellism of global committer
>> > > > must be one. By design it would be used in several cases:
>> > > > 1. writer -> committer: for normal sinks that write pending data in
>> > > writer
>> > > > and commit pending data in committer.
>> > > > 2. writer -> global committer: for sinks require the committer's
>> > > > parallelism be 1, like iceberg sink which
>> > > > relies on optimistic lock to commit data hopes to reduce the
>> conflicts.
>> > > > 3. writer -> committer -> global committer: for sinks that also
>> need to
>> > > > write meta data (like _SUCCESS file
>> > > > or add record in hive metastore) in global committer.
>> > > >
>> > > > The third case would cause the cascade commit problem. To overcome
>> this
>> > > > problem, we would like to
>> > > > 1. Previously the global committer also support writing metadata
>> with
>> > > 2pc.
>> > > > Now we disable this mode
>> > > > and always rely on the property that writing metadata is
>> repeatable.
>> > > With
>> > > > this limitation there should no
>> > > > need of cascade commit, but the committer would still emit records
>> in
>> > > > notifyCheckpointComplete.
>> > > > 2. We further move global committer in the case 3 to the operator
>> > > > coordinator. Then the committer would
>> > > > not need to emit records, but communicates with its operator
>> > > coordinator.
>> > > >
>> > > > One core issue for using the OperatorCoordinator is how to keep the
>> > > > communication between the operator
>> > > > and the OperatorCoordinator exactly-once. Since the message is
>> always
>> > > from
>> > > > the oeprator to the OperatorCoordinator
>> > > > in this case, we would only need to bookkeep the message sent
>> between
>> > > the
>> > > > OperatorCoordinator takes snapshot and
>> > > > the Operator takes snapshot in the state of the Operator.
>> > > >
>> > > > On how to achieve the change in detail we would still need some
>> think,
>> > > it
>> > > > currently seems we would have to had
>> > > > some modification to the current new sink api.
>> > > >
>> > > >
>> > > >
>> > > > B. Regarding the stop-with-savepoint --drain
>> > > >
>> > > > Very thanks @Piotr for the further explanation and now I realize I
>> have
>> > > > understand wrongly for the semantics of
>> > > > stop-with-savepoint --drain. Now I think that the problem should be
>> we
>> > > > should also include the records produced in
>> > > > `endOfInput()` and `close()` also in the last savepoint, am I
>> correct?
>> > > If
>> > > > so, it seems we still have some undetermined options for
>> > > > the lifecycle of the operator, like in Kezhu's proposal the close()
>> > > happens
>> > > > at last, but it seems close() might also emit records (
>> > > > so now the operator are closed with op1's close() -> op2's
>> endOfInput()
>> > > ->
>> > > > op2's close() -> op3's endOfinput -> ...) ?
>> > > >
>> > > > And on the other side, as Kezhu has also proposed, perhapse we
>> might
>> > > have
>> > > > the stop-with-savepoint --drain and normal exit in the same
>> process,
>> > > > but have slightly difference in that stop-with-savepoint --drain
>> would
>> > > wait
>> > > > for a specific savepoint and in normal exit, the operator
>> > > > might wait for arbitrary checkpoint. If we could achieve not
>> emitting
>> > > > records in notifyCheckpointComplete, stop-with-savepoint --drain
>> could
>> > > > be done with one savepoint, and for the normal exit, the operator
>> would
>> > > not
>> > > > need to wait for other slow operators to exit.
>> > > >
>> > > > Best,
>> > > > Yun
>> > > >
>> > > >
>> > > >
>> > > > ------------------Original Mail ------------------
>> > > > *Sender:*Kezhu Wang <ke...@gmail.com>
>> > > > *Send Date:*Thu Feb 25 15:11:53 2021
>> > > > *Recipients:*Flink Dev <de...@flink.apache.org>, Piotr Nowojski <
>> > > > piotr.nowojski@gmail.com>
>> > > > *CC:*Guowei Ma <gu...@gmail.com>, jingsonglee0@gmail.com <
>> > > > jingsonglee0@gmail.com>
>> > > > *Subject:*Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After
>> Tasks
>> > > > Finished
>> > > >
>> > > > > Hi all, thanks for driving this and especially Piotr for
>> re-active
>> > > this
>> > > > >
>> > > > > thread.
>> > > > >
>> > > > >
>> > > > >
>> > > > > First, for `notifyCheckpointComplete`, I have strong preference
>> > > towards
>> > > > >
>> > > > > "shut down the dataflow
>> > > > >
>> > > > > pipeline with one checkpoint in total", so I tend to option
>> dropping
>> > > > "send
>> > > > >
>> > > > > records" from
>> > > > >
>> > > > > `notifyCheckpointComplete` for next level committing in pipeline,
>> if
>> > > we
>> > > > >
>> > > > > ever support it. Without this,
>> > > > >
>> > > > > we are unable to stop a pipeline manually with all results
>> > > materialized.
>> > > > >
>> > > > >
>> > > > >
>> > > > > Second, for shutdown unification of `stop-with-savepoint --drain`
>> and
>> > > > >
>> > > > > FLIP-147, I draft following
>> > > > >
>> > > > > phase based on emerging proposals(mainly by Piotr and Till) in
>> this
>> > > > thread
>> > > > >
>> > > > > and FLINK-21133.
>> > > > >
>> > > > >
>> > > > >
>> > > > > ```java
>> > > > >
>> > > > > // StreamTask termination phase
>> > > > >
>> > > > > finish()(Call StreamOperator.finish in chaining order)
>> > > > >
>> >
>> > > > > advanceToEndOfEventTime()(nop if for no-source inputs, this could
>> also
>> > > be
>> > > > >
>> > > > > done in input processor or finish ?)
>> > > > >
>> > > > > if (there-is-a-pending-terminate-savepoint) {
>> > > > >
>> > > > > triggerBarrierForDownStream();
>> > > > >
>> > > > > waitCheckpointComplete();
>> > > > >
>> > > > > } else if (require-two-phase-commit-in-shutdown) {
>> > > > >
>> > > > > waitFinalCheckpoint();
>> > > > >
>> > > > > waitCheckpointComplete();
>> > > > >
>> > > > > }
>> > > > >
>> > > > > // TODO: Ignore/Cancel all checkpoints from this point(or hit to
>> > > > >
>> > > > > coordinator in final checkpoint response ?).
>> > > > >
>> > > > > close();
>> > > > >
>> > > > > ```
>> > > > >
>> > > > >
>> > > > >
>> > > > > Branches in `if` could reside in different code paths, but the
>> effect
>> > > > >
>> > > > > should be similar to above.
>> > > > >
>> > > > > The divergence reflects fact that "stop-with-savepoint --drain"
>> is
>> > > > >
>> > > > > triggered already while we need
>> > > > >
>> > > > > to wait for final checkpoint in natural input exhausting.
>> > > > >
>> > > > >
>> > > > >
>> > > > > With carefully chosen default functions, we will not break
>> existing
>> > > > >
>> > > > > interfaces.
>> > > > >
>> > > > > * `StreamOperator.finish`
>> > > > >
>> > > > > * `RichFunction.finish`
>> > > > >
>> > > > >
>> > > > >
>> > > > > For existing codes:
>> > > > >
>> > > > > 1. "stop-with-savepoint" with no migration, it should behaves as
>> > > before.
>> > > > >
>> > > > > 2. "require-two-phase-commit-in-shutdown" evaluates to false and
>> no
>> > > > >
>> > > > > migration, it is same as before.
>> > > > >
>> > > > > 3. "require-two-phase-commit-in-shutdown" evaluates to true and
>> no
>> > > > >
>> > > > > migration, records in `close` will
>> > > > >
>> > > > > lose.
>> > > > >
>> > > > >
>> > > > >
>> > > > > For case#3:
>> > > > >
>> > > > > A. No external materialization in `StreamOperator.close`: Nothing
>> to
>> > > > lose.
>> > > > >
>> > > > > B. External materialization only in `StreamOperator.close`: I
>> don't
>> > > think
>> > > > >
>> > > > > Flink ever claimed that there is
>> > > > >
>> > > > > any guarantee for this situation.
>> > > > >
>> > > > > C. External materialization in `notifyCheckpointComplete` from
>> > > > >
>> > > > > `StreamOperator.close`: But due to fact that
>> > > > >
>> > > > > `notifyCheckpointComplete` was ignored after operator
>> > > > >
>> > > > > closed(FLINK-16383), so there will be no external
>> > > > >
>> > > > > materialization to lose.
>> > > > >
>> > > > >
>> > > > >
>> > > > > Then, we could recommend users to migrate possible
>> > > > “end-of-stream-flushing”
>> > > > >
>> > > > > from “close" to “finish”.
>> > > > >
>> > > > >
>> > > > >
>> > > > > Best,
>> > > > >
>> > > > > Kezhu Wang
>> > > > >
>> > > > >
>> > > > >
>> > > > >
>> > > > >
>> > > > > On February 24, 2021 at 23:47:36, Piotr Nowojski (
>> > > > piotr.nowojski@gmail.com
>> > > > > )
>> > > > >
>> > > > > wrote:
>> > > > >
>> > > > >
>> > > > >
>> > > > > Thanks for the reponses Guowei and Yun,
>> > > > >
>> > > > >
>> > > > >
>> > > > > Could you elaborate more/remind me, what does it mean to replace
>> > > emitting
>> > > > >
>> >
>> > > > > results from the `notifyCheckpointComplete` with
>> `OperatorCoordinator`
>> > > > >
>> > > > > approach?
>> > > > >
>> > > > >
>> > > > >
>> > > > > About the discussion in FLINK-21133 and how it relates to
>> FLIP-147.
>> > > You
>> > > > are
>> > > > >
>> >
>> > > > > right Yun gao, that in case of `stop-with-savepoint --drain` the
>> whole
>> > > > job
>> > > > >
>> > > > > finishes, while in FLIP-147 some of the sources can work for
>> > > arbitrarily
>> > > > >
>> >
>> > > > > long time after others have already finished. However from the
>> runtime
>> > > > >
>> > > > > perspective, especially how to close a Task and it's operators,
>> there
>> > > is
>> > > > no
>> > > > >
>> > > > > difference between those two cases. In both cases, we need to end
>> > > input,
>> > > > >
>> >
>> > > > > shut down processing time timers and commit external side effects
>> (two
>> > > > >
>> > > > > phase commit support) before exiting the task. That's how the
>> > > discussion
>> > > > >
>> > > > > about the behaviour of "stop-with-savepoint" was connected with
>> > > FLIP-147.
>> > > > >
>> > > > >
>> > > > >
>> > > > > Currently on master, "stop-with-savepoint --drain" drains/flushes
>> > > > buffered
>> > > > >
>> >
>> > > > > records and deals correctly with timers, but all of that happens
>> AFTER
>> > > > >
>> > > > > savepoint was completed. So any records flushed from the
>> operators
>> > > during
>> > > > >
>> > > > > endOfInput/close/shutting down processing timers are never
>> committed
>> > > to
>> > > > >
>> > > > > external systems. This is exactly the same problem as the "two
>> phase
>> > > > >
>> >
>> > > > > commit" problem of FLIP-147, that should have the same solution
>> and it
>> > > > >
>> >
>> > > > > should be solved at the same time. For example if we go with the
>> *very
>> > > > >
>> > > > > undesirable* "one closed operator per one completed
>> > > > checkpoint/savepoint",
>> > > > >
>> > > > > in both cases CheckpointCoordinator, Scheduler and Task would
>> need to
>> > > > keep
>> > > > >
>> > > > > the task alive and keep triggering checkpoints for that task,
>> until
>> > > all
>> > > > >
>> > > > > operators in the operator chain are closed (one closed operator
>> per
>> > > one
>> > > > >
>> > > > > completed checkpoint).
>> > > > >
>> > > > >
>> > > > >
>> > > > > Piotrek
>> > > > >
>> > > > >
>> > > > >
>> > > > >
>> > > > >
>> > > > >
>> > > > >
>> > > > > śr., 24 lut 2021 o 11:30 Yun Gao napisał(a):
>> > > > >
>> > > > >
>> > > > >
>> > > > > > Hi Till, Guowei,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > Very thanks for initiating the disucssion and the deep
>> thoughts!
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > For the notifyCheckpointComplete, I also agree we could try to
>> avoid
>> > > > >
>> > > > > > emitting new records in notifyCheckpointComplete via using
>> > > > >
>> > > > > > OperatorCoordinator
>> > > > >
>> > > > > > for new sink API. Besides, the hive sink might also need some
>> > > > >
>> > > > > modification
>> > > > >
>> > > > > > for it also emits records in notifyCheckpointComplete.
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > For unifying the process of stopping with savepoint and
>> finished due
>> > > to
>> > > > >
>> > > > > > all records
>> > > > >
>> > > > > > are processed, I also agree with that unifying would always be
>> > > better
>> > > > if
>> > > > >
>> > > > > > we could achieve,
>> > > > >
>> > > > > > but I'm still not fully catch up with the implementation: Based
>> on
>> > > the
>> > > > >
>> > > > > > discussion in FLINK-21133,
>> > > > >
>> >
>> > > > > > my understanding is that for stopping with savepoint, now we
>> want to
>> > > > >
>> > > > > first
>> > > > >
>> > > > > > stop the source, then we
>> > > > >
>> > > > > > trigger a savepoint, and after the source received
>> > > > >
>> > > > > > notifyCheckpointComplete, the source would
>> > > > >
>> >
>> > > > > > start emitting EndOfPartitionEvent to finish the job, am I
>> correct ?
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > For normal finish, a difference to me might be if we have
>> multiple
>> > > > >
>> > > > > > sources, we could not guarantee
>> > > > >
>> > > > > > when the sources are to finish. We might have one source run
>> one 1
>> > > > minute
>> > > > >
>> > > > > > and another one run for
>> > > > >
>> >
>> > > > > > 1 hour. To unify with the process with stop with savepoint, we
>> might
>> > > > need
>> > > > >
>> > > > > > to hold the fast source until
>> > > > >
>> > > > > > all the sources are finished? An coordinator would be
>> introduced to
>> > > > count
>> > > > >
>> > > > > > the number of sources
>> > > > >
>> >
>> > > > > > runing and trigger the final savepoint / checkpoint. For the
>> extreme
>> > > > >
>> > > > > > cases, if we have both bounded and
>> > > > >
>> > > > > > unbounded sources, we might only count how much bounded source
>> are
>> > > > >
>> > > > > > remaining ? And if all the bounded
>> > > > >
>> > > > > > sources are finished we would trigger the special checkpoint.
>> After
>> > > all
>> > > > >
>> > > > > > the bounded part of the graph are
>> > > > >
>> > > > > > finished, the the remaining part could still do checkpoint and
>> > > commit
>> > > > >
>> > > > > data
>> > > > >
>> > > > > > with FLIP-147.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > Best,
>> > > > >
>> > > > > > Yun
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > ------------------Original Mail ------------------
>> > > > >
>> > > > > > Sender:Guowei Ma
>> > > > >
>> > > > > > Send Date:Wed Feb 24 17:35:36 2021
>> > > > >
>> > > > > > Recipients:dev
>> > > > >
>> > > > > > CC:Arvid Heise
>> > > > >
>> > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
>> > > > Finished
>> > > > >
>> > > > > > Hi, Till
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > Thank you very much for your careful consideration
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > *1. Emit records in `NotifyCheckpointComplete`.*
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > Sorry for making you misunderstanding because of my expression.
>> I
>> > > just
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > want to say the current interface does not prevent users from
>> doing
>> > > it.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > From the perspective of the new sink api, we might not depend
>> on
>> > > > emitting
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > records in `NotifyCheckpointComplete`, like using
>> > > `OperatorCoordinator`
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > instead.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > *2. What does the FLIP-147 guarantee?*I think initially this
>> FLIP
>> > > want
>> > > > to
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > achieve two targets:
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > 1. Tasks/Operators exit correctly (as you mentioned the
>> lifecycle of
>> > > a
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > Task/StreamTask/StreamOperator.).
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > 2. Continue to trigger checkpoint after some tasks for mixed
>> jobs.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > I think the first thing is related to the discussion in
>> FLINK-21133.
>> > > > If I
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > understand correctly, in addition to supporting the tasks /
>> > > operators
>> > > > to
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > exit correctly, now we also want to unify the process of the
>> tasks
>> > > and
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > operators for savepoint / finish.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > I think the second thing is orthogonal to the FLINK-21133
>> because
>> > > there
>> > > > >
>> > > > > are
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > topologies that have both the bounded and unbounded input.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > *3. How to unify the operator exit process of FLIP-147 with
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > stop-with-savepoint?*
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > I am not very sure about how to do it yet. But if I understand
>> the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > discussion in the jira correctly it needs to introduce some
>> logic
>> > > into
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > `CheckpointCoordinator`, which responses for triggering “the
>> unified
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > operator exit process”. Am I correct?
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > Best,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > Guowei
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann wrote:
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > Thanks for the explanation Yun and Guowei. I have to admit
>> that I
>> > > do
>> > > > >
>> > > > > not
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > fully understand why this is strictly required but I think
>> that we
>> > > > are
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > touching two very important aspects which might have far
>> fetching
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > consequences for how Flink works:
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > 1) Do we want to allow that multiple checkpoints are required
>> to
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > materialize results?
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > 2) Do we want to allow to emit records in
>> > > notifyCheckpointComplete?
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > For 1) I am not sure whether this has been discussed within
>> the
>> > > > >
>> > > > > community
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > sufficiently. Requiring multiple checkpoints to materialize a
>> > > result
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > because of multi level committers has the consequence that we
>> > > > increase
>> > > > >
>> > > > > > the
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > latency from checkpoint interval to #levels * checkpoint
>> interval.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > Moreover, having to drain the pipeline in multiple steps,
>> would
>> > > break
>> > > > >
>> > > > > the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > stop-with-savepoint --drain because which savepoint do you
>> report
>> > > to
>> > > > >
>> > > > > the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > user?
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > For 2) allowing to send records after the final
>> > > > >
>> > > > > notifyCheckpointComplete
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > will effectively mean that we need to shut down a topology in
>> > > > multiple
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > steps (in the worst case one operator per checkpoint). This
>> would
>> > > be
>> > > > a
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > strong argument for not allowing this to me. The fact that
>> users
>> > > can
>> > > > >
>> > > > > send
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > records after the notifyCheckpointComplete is more by
>> accident
>> > > than
>> > > > by
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > design. I think we should make this a very deliberate
>> decision and
>> > > in
>> > > > >
>> > > > > > doubt
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > I would be in favour of a more restrictive model unless there
>> is a
>> > > > very
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > good reason why this should be supported.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > Taking also the discussion in FLINK-21133 [1] into account,
>> it
>> > > seems
>> > > > to
>> > > > >
>> > > > > > me
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > that we haven't really understood what kind of guarantees we
>> want
>> > > to
>> > > > >
>> > > > > give
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > to our users and how the final checkpoint should exactly
>> work. I
>> > > > >
>> > > > > > understand
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > that this is not included in the first scope of FLIP-147 but
>> I
>> > > think
>> > > > >
>> > > > > this
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > is so important that we should figure this out asap. Also
>> because
>> > > the
>> > > > >
>> > > > > > exact
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > shut down behaviour will have to be aligned with the
>> lifecycle of
>> > > a
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > Task/StreamTask/StreamOperator. And last but not least
>> because
>> > > other
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > features such as the new sink API start building upon a shut
>> down
>> > > > model
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > which has not been fully understood/agreed upon.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > [1] https://issues.apache.org/jira/browse/FLINK-21133
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > Cheers,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > Till
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma wrote:
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > Thanks Yun for the detailed explanation.
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > A simple supplementary explanation about the sink case:
>> Maybe we
>> > > > >
>> > > > > could
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > use
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > `OperatorCoordinator` to avoid sending the element to the
>> > > > downstream
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > operator.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > But I agree we could not limit the users not to emit
>> records in
>> > > the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > `notiyCheckpointComplete`.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > Best,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > Guowei
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > wrote:
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > Hi all,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > I'd like to first detail the issue with emitting records
>> in
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > notifyCheckpointComplete for context. For specific usage,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > an example would be for sink, it might want to write some
>> > > > metadata
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > after
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > all the transactions are committed
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > (like write a marker file _SUCCESS to the output
>> directory).
>> > > This
>> > > > >
>> > > > > > case
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > is
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > currently supported via the two level
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > committers of the new sink API: when received
>> endOfInput(),
>> > > the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > Committer
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > wait for another checkpoint to
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > commits all the pending transactions and emit the list of
>> > > files
>> > > > to
>> > > > >
>> > > > > > the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > GlobalCommitter. The GlobalCommitter
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > would wait for another checkpoint to also write the
>> metadata
>> > > with
>> > > > >
>> > > > > 2pc
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > (Although sometimes 2pc is not needed
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > for writing metadata, it should be only an optimization
>> and
>> > > still
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > requires
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > the Committer do commit before
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > notifying the global Committer. Also another note is
>> > > > >
>> > > > > GlobalCommitter
>> > > > >
>> > > > > > is
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > also added for some other cases
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > like some sinks want an commiter with dop = 1, like
>> > > IceBergSink).
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > However, a more general issue to me is that currently we
>> do
>> > > not
>> > > > >
>> > > > > limit
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > users to not emit records in
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > notifyCheckpointComplete in the API level. The sink case
>> could
>> > > be
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > viewed
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > as a special case, but in addition
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > to this one, logically users could also implement their
>> own
>> > > cases
>> > > > >
>> > > > > > that
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > emits records in notifyCheckpointComplete.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > Best,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > Yun
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > ------------------Original Mail ------------------
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > Sender:Arvid Heise
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > Send Date:Fri Feb 12 20:46:04 2021
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > Recipients:dev
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > CC:Yun Gao
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After
>> > > Tasks
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > Finished
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > Hi Piotr,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > Thank you for raising your concern. Unfortunately, I do
>> not
>> > > have
>> > > > a
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > better
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > idea than doing closing of operators intermittently with
>> > > > >
>> > > > > checkpoints
>> > > > >
>> > > > > > (=
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > multiple last checkpoints).
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > However, two ideas on how to improve the overall user
>> > > experience:
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > 1. If an operator is not relying on
>> notifyCheckpointComplete,
>> > > we
>> > > > >
>> > > > > can
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > close
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > it faster (without waiting for a checkpoint). In general,
>> I'd
>> > > > >
>> > > > > assume
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > that
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > almost all non-sinks behave that way.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > 2. We may increase the checkpointing frequency for the
>> last
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > checkpoints.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > We
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > need to avoid overloading checkpoint storages and task
>> > > managers,
>> > > > >
>> > > > > but
>> > > > >
>> > > > > > I
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > assume the more operators are closed, the lower the
>> > > checkpointing
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > interval
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > can be.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > For 1, I'd propose to add (name TBD):
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > default boolean StreamOperator#requiresFinalCheckpoint()
>> {
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > return true;
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > }
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > This means all operators are conservatively (=slowly)
>> closed.
>> > > For
>> > > > >
>> > > > > > most
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > operators, we can then define their behavior by
>> overriding in
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > AbstractUdfStreamOperator
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > @Override
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > boolean
>> AbstractUdfStreamOperator#requiresFinalCheckpoint() {
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > return userFunction instanceof CheckpointListener;
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > }
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > This idea can be further refined in also adding
>> > > > >
>> > > > > > requiresFinalCheckpoint
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > to
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > CheckpointListener to exclude all operators with UDFs
>> that
>> > > > >
>> > > > > implement
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > CheckpointListener but do not need it for 2pc.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > @Override
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > boolean
>> AbstractUdfStreamOperator#requiresFinalCheckpoint() {
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > return userFunction instanceof CheckpointListener &&
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > ((CheckpointListener)
>> userFunction).requiresFinalCheckpoint();
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > }
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > That approach would also work for statebackends/snapshot
>> > > > strategies
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > that
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > require some 2pc.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > If we can contain it to the @PublicEvolving
>> StreamOperator, it
>> > > > >
>> > > > > would
>> > > > >
>> > > > > > be
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > better of course.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > Best,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > Arvid
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > wrote:
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > Hey,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > I would like to raise a concern about implementation of
>> the
>> > > > final
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > > checkpoints taking into account operators/functions
>> that are
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > implementing
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > two phase commit (2pc) protocol for exactly-once
>> processing
>> > > > with
>> > > > >
>> > > > > > some
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > external state (kept outside of the Flink). Primarily
>> > > > >
>> > > > > exactly-once
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > sinks.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > > First of all, as I understand it, this is not planned
>> in the
>> > > > >
>> > > > > first
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > version
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > of this FLIP. I'm fine with that, however I would
>> strongly
>> > > > >
>> > > > > > emphasize
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > this
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > in every place we will be mentioning FLIP-147 efforts.
>> This
>> > > is
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > because
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > me,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > as a user, upon hearing "Flink supports checkpointing
>> with
>> > > > >
>> > > > > bounded
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > inputs"
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > I would expect 2pc to work properly and to commit the
>> > > external
>> > > > >
>> > > > > side
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > effects
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > upon finishing. As it is now, I (as a user) would be
>> > > surprised
>> > > > >
>> > > > > > with a
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > silent data loss (of not committed trailing data). This
>> is
>> > > > just a
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > remark,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > that we need to attach this warning to every blog
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > post/documentation/user
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > mailing list response related to "Support Checkpoints
>> After
>> > > > Tasks
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > Finished". Also I would suggest to prioritize the
>> follow up
>> > > of
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > supporting
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > 2pc.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > Secondly, I think we are missing how difficult and
>> > > problematic
>> > > > >
>> > > > > will
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > be
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > 2pc
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > support with the final checkpoint.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > For starters, keep in mind that currently 2pc can be
>> > > > implemented
>> > > > >
>> > > > > by
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > users
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > > using both `@Public` APIs as functions and
>> `@PublicEvolving`
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > operators
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > in
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > any place in the job graph. It's not limited to only
>> the
>> > > sinks.
>> > > > >
>> > > > > For
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > > example users could easily implement the `AsynFunction`
>> (for
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > `AsyncWaitOperator`) that is using 2pc based on the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > `CheckpointListener`
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > interface. I'm not saying it's common, probably just a
>> tiny
>> > > > >
>> > > > > > minority
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > of
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > users are doing that (if any at all), but nevertheless
>> > > that's
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > possible
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > and
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > currently (implicitly?) supported in Flink.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > Next complication is the support of bounded streams
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > (`BoundedOneInput`
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > or
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > `BoundedMultiInput` interfaces) and the
>> closing/shutdown
>> > > > >
>> > > > > procedure
>> > > > >
>> > > > > > of
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > operators. Currently it works as follows:
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > 0. Task receives EndOfPartitionEvent (or source
>> finishes)
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > > 1. `endOfInput` is called on the first operator in the
>> chain
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > 2. We quiesce the processing timers
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > >
>> (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
>> > > > for
>> > > > >
>> > > > > > the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > first
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > operator, so no new timers will be triggered
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > 3. We wait for the already fired timers to finish
>> executing
>> > > > >
>> > > > > > (spinning
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > mailbox loop)
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > 4. We are closing the first operator
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > 5. We go to the next (second) operator in the chain and
>> > > repeat
>> > > > >
>> > > > > the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > steps
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > 1.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > to 5.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > This is because operators can emit data after
>> processing
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > `endOfInput`,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > from
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > timers, async mailbox actions and inside the `close`
>> method
>> > > > >
>> > > > > itself.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > > Now the problem is to support the final checkpoint with
>> 2pc,
>> > > we
>> > > > >
>> > > > > > need
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > trigger `snapshotState` and `notifyCheckpointComplete`
>> call
>> > > at
>> > > > >
>> > > > > the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > very
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > > least only after `endOfInput` call on the operator.
>> Probably
>> > > > the
>> > > > >
>> > > > > > best
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > place
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > would be in between steps 3. and 4. However that means,
>> we
>> > > > would
>> > > > >
>> > > > > be
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > forced
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > to wait for steps 1. to 3. to finish, then wait for a
>> next
>> > > > >
>> > > > > > checkpoint
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > to
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > trigger AND complete, before finally closing the head
>> > > operator,
>> > > > >
>> > > > > and
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > only
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > then we can start closing the next operator in the
>> chain:
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > 0. Task receives EndOfPartitionEvent (or source
>> finishes)
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > > 1. `endOfInput` is called on the first operator in the
>> chain
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > 2. We quiesce the processing timers
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > >
>> (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
>> > > > for
>> > > > >
>> > > > > > the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > first
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > operator, so no new timers will be triggered
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > 3. We wait for the already fired timers to finish
>> executing
>> > > > >
>> > > > > > (spinning
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > mailbox loop)
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > *3b. We wait for one more checkpoint to trigger and for
>> the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > `notifyCheckpointComplete` RPC.*
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > 4. We are closing the first operator
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > 5. We go to the next (second) operator in the chain and
>> > > repeat
>> > > > >
>> > > > > the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > steps
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > 1.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > to 5.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > That means, we can close one operator per successful
>> > > > checkpoint.
>> > > > >
>> > > > > To
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > close
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > 10 operators, we would need 10 successful checkpoints.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > I was thinking about different approaches to this
>> problem,
>> > > and
>> > > > I
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > couldn't
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > find any viable ones. All I could think of would break
>> the
>> > > > >
>> > > > > current
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > `@Public` API and/or would be ugly/confusing for the
>> users.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > For example a relatively simple solution, to introduce
>> a
>> > > > >
>> > > > > `preClose`
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > or
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > `flush` method to the operators, with a contract that
>> after
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > `flush`, operators would be forbidden from emitting
>> more
>> > > > records,
>> > > > >
>> > > > > > so
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > that
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > we can replace step 4. with this `flush` call, and then
>> > > having
>> > > > a
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > single
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > > checkpoint to finish 2pc for all of the operators
>> inside the
>> > > > >
>> > > > > chain,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > doesn't
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > work. Sheer fact of adding this `flush` method and
>> changing
>> > > the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > contract
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > would break the current API and Yun Gao has pointed out
>> to
>> > > me,
>> > > > >
>> > > > > that
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > we
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > either already support, or want to support operators
>> that
>> > > are
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > emitting
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > records from within the `notifyCheckpointComplete`
>> call:
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > Yun Gao:
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > like with the new sink api there might be writer ->
>> > > committer
>> > > > >
>> > > > > ->
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > global
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > committer, the committer would need to wait for the
>> last
>> > > > >
>> > > > > checkpoint
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > to
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > commit
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > the last piece of data, and after that it also need
>> to
>> > > emit
>> > > > the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > list
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > of
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > transactions get committed to global committer to do
>> some
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > finalization
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > logic.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > So it wouldn't solve the problem (at least not fully).
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > I don't know if anyone has any better ideas how to
>> solve
>> > > this
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > problem?
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > Piotrek
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > pt., 15 sty 2021 o 14:57 Yun Gao
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > napisał(a):
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > Hi Aljoscha,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > I think so since we seems to do not have other
>> divergence
>> > > and
>> > > > >
>> > > > > new
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > objections now. I'll open the vote then. Very thanks!
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > Best,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > Yun
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > >
>> > > > >
>> > > > > >
>> ------------------------------------------------------------------
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > From:Aljoscha Krettek
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > Send Time:2021 Jan. 15 (Fri.) 21:24
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > To:dev
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints
>> After
>> > > > Tasks
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > Finished
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > Thanks for the summary! I think we can now move
>> towards a
>> > > > >
>> > > > > [VOTE]
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > thread,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > right?
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > On 2021/01/15 13:43, Yun Gao wrote:
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > >1) For the problem that the "new" root task
>> coincidently
>> > > > >
>> > > > > > finished
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > > > >before getting triggered successfully, we have
>> listed two
>> > > > >
>> > > > > > options
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > in
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > >the FLIP-147[1], for the first version, now we are
>> not
>> > > tend
>> > > > to
>> > > > >
>> > > > > > go
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > with
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > >the first option that JM would re-compute and
>> re-trigger
>> > > new
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > sources
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > >when it realized some tasks are not triggered
>> > > successfully.
>> > > > >
>> > > > > This
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > option
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > >would avoid the complexity of adding new PRC and
>> > > duplicating
>> > > > >
>> > > > > > task
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > >states, and in average case it would not cause too
>> much
>> > > > >
>> > > > > > overhead.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> >
>> > > > > > > > > > > You wrote "we are *not* tend to go with the first
>> option",
>> > > > but
>> > > > >
>> > > > > I
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > think
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > you meant wo write "we tend to *now* go with the
>> first
>> > > > option",
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > right?
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > That's also how it is in the FLIP, I just wanted to
>> > > clarify
>> > > > for
>> > > > >
>> > > > > > the
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > > mailing list.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > >
>> > > >
>> > >
>> > >
>> >
>> >
>> >
>>
>>

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

Posted by Till Rohrmann <tr...@apache.org>.
Thanks a lot for all your input. To sum up the discussion so far:

## Final checkpoints

We currently agree on favouring a single final checkpoint which can shut
down the topology. In order to support this we need to be able to create a
checkpoint after an operator has finished producing results.

If we want to send checkpoint barriers through the topology this means that
a task must not close the network connection when it sees "logical end of
data". Instead, on the "logical end of data" the contained operator should
flush all of its records. This means that we need to introduce a new event
"logical end of data" and API calls to signal an operator that it should
flush its data and that it should shut down.

Given the available methods, `endInput` could be used for signalling the
"logical end of data" and `dispose` for shutting the operator down. A task
will only shut down and send an "EndOfPartitionEvent" which closes the TCP
connection if all of its inputs have shut down and if it has completed a
final checkpoint.

## Global commits

Now a somewhat related but also orthogonal issue is how to support a global
commit. A global commit is a commit where the external artefacts of a
checkpoint become visible all at once. The global commit should be
supported for streaming as well as batch executions (it is probably even
more important for batch executions). In general, there could be different
ways of implementing the global commit mechanism:

1. Collect global commit handles on the JM and run the global commit action
on the JM
2. Collect global commit handles in a parallelism 1 operator which performs
the global commit action

Approach 2. would probably require to be able to send records from the
snapshotState() method which would be the global commit handles. Both
approaches would have to persist some kind of information in the checkpoint
which allows redoing the global commit operation in case of a failover.
Therefore, for approach 1. it would be required that we send the global
commit handles to the JM from the snapshotState() method and not the
notifyCheckpointComplete().

A related question is in which order to execute the local and global commit
actions:

1. Unspecified order
2. First local commits and then global commits

Option 1. would be easier to implement and might already be good enough for
most sinks.

I would suggest treating final checkpoints and global commits as two
related but separate things. I think it is fine to first try to solve the
final checkpoints problem and then to tackle the global commits. This will
help to decrease the scope of each individual feature.

Cheers,
Till

On Fri, Mar 5, 2021 at 5:12 AM Yun Gao <yu...@aliyun.com> wrote:

> Hi Piotr,
>
> Very thanks for the suggestions and thoughts!
>
> > Is this a problem? Pipeline would be empty, so EndOfPartitionEvent would
> be traveling very quickly.
>
> No, this is not a problem, sorry I have some wrong thoughts here,
> initially in fact I'm thinking on this issue raised by
> @kezhu:
>
> > Besides this, will FLIP-147 eventually need some ways to decide whether
> an operator need final checkpoint
> @Yun @Guowei ?  @Arvid mentions this in earlier mail.
>
> For this issue itself, I'm still lean towards we might still need it, for
> example, suppose we have a job that
> do not need to commit anything on finished, then it do not need to wait
> for checkpoint at all for normal
> finish case.
>
> > Yes, but aren't we doing it right now anyway?
> `StreamSource#advanceToEndOfEventTime`?
>
> Yes, we indeed have advancedEndOfEventTime for both legacy and new
> sources, sorry for the overlook.
>
> > Is this the plan? That upon recovery we are restarting all operators,
> even those that have already finished?
> Certainly it's one of the possibilities.
>
> For the first version we would tend to use this way since it is easier to
> implement, and we should always need
> to consider the case that tasks are started but operators are finished
> since there might be also tasks with part
> of operators finished. For the long run I think we could continue to
> optimize it via not restart the finished tasks
> at all.
>
> > Keep in mind that those are two separate things, as I mentioned in a
> previous e-mail:
> > > II. When should the `GlobalCommitHandle` be created? Should it be
> returned from `snapshotState()`, `notifyCheckpointComplete()` or somewhere
> else?
> > > III. What should be the ordering guarantee between global commit and
> local commit, if any? Actually the easiest to implement would be undefined,
> but de facto global commit happening before local commits (first invoke > `notifyCheckpointComplete()`
> on the `OperatorCoordinator` and either after or in parallel send
> `notifyCheckpointComplete()` RPCs). As far as I can tell, undefined order
> should work for the use cases that I'm aware of.
> >
> > We could create the `GlobalCommitHandle` in
> `StreamOperator#snapshotState()`, while we could also ensure that
> `notifyCheckpointComplete()` is called on the `OperatorCoordinator` AFTER
> all of the operators have successfully > processed
> `notifyCheckpointComplete()`. This would be more difficult to implement,
> hence I would prefer "undefined" behaviour here, but it's probably possible.
>
> Very thanks for the further explanation, and I also totally agree with
> that the two are separate and we could think on them
> distinctly. Regarding the order, I would still tend to we support the
> ordered case, since the sinks' implementation seem to depend
> on this functionality.
>
> Best,
>  Yun
>
> ------------------------------------------------------------------
> From:Piotr Nowojski <pn...@apache.org>
> Send Time:2021 Mar. 4 (Thu.) 22:56
> To:Kezhu Wang <ke...@gmail.com>
> Cc:Till Rohrmann <tr...@apache.org>; Guowei Ma <gu...@gmail.com>;
> dev <de...@flink.apache.org>; Yun Gao <yu...@aliyun.com>;
> jingsonglee0@gmail.com <ji...@gmail.com>
> Subject:Re: Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> Finished
>
> Hi Yun and Kezhu,
>
> > 1. We might introduce a new type of event to notify the endOfInput()
> though the graph first, and then A/B waits for the final
> checkpoint, then A emit EndOfPartitionEvent to exit all the tasks as now.
>
> As I mentioned in one of the PRs, I would opt for this solution.
>
> >  if we go towards 1, the tasks would still need to exit from the source,
> and if we go towards 2/3, we could be able to allow these
> tasks to finish first.
>
> Is this a problem? Pipeline would be empty, so EndOfPartitionEvent would
> be traveling very quickly.
>
> > should we also need to do it for normal exit
>
> Yes, but aren't we doing it right now anyway?
> `StreamSource#advanceToEndOfEventTime`?
>
> > If so, since now for recovery after some tasks finished we would first
> start all the tasks and stop the finished tasks directly
>
> Is this the plan? That upon recovery we are restarting all operators, even
> those that have already finished? Certainly it's one of the possibilities.
>
> > For example, the global committer handler might be write meta store for
> FileSystem/Hive sinks, and these should happen after all the pending
> > files are renamed to the final ones, otherwise the downstream jobs might
> miss some files if they relies on the meta store to identify ready
> partitions.
> > Thus we would have to emit the global-committer-handler after
> notifyCheckpointComplete. But since we could be able to know the list of
> files
> > to rename in snapshotState(), we could create the
> global-committer-handler and store them there.
>
> Keep in mind that those are two separate things, as I mentioned in a
> previous e-mail:
> > II. When should the `GlobalCommitHandle` be created? Should it be
> returned from `snapshotState()`, `notifyCheckpointComplete()` or somewhere
> else?
> > III. What should be the ordering guarantee between global commit and
> local commit, if any? Actually the easiest to implement would be undefined,
> but de facto global commit happening before local commits (first invoke
> `notifyCheckpointComplete()` on the `OperatorCoordinator` and either after
> or in parallel send `notifyCheckpointComplete()` RPCs). As far as I can
> tell, undefined order should work for the use cases that I'm aware of.
>
> We could create the `GlobalCommitHandle` in
> `StreamOperator#snapshotState()`, while we could also ensure that
> `notifyCheckpointComplete()` is called on the `OperatorCoordinator` AFTER
> all of the operators have successfully processed
> `notifyCheckpointComplete()`. This would be more difficult to implement,
> hence I would prefer "undefined" behaviour here, but it's probably possible.
>
> Kezhu:
>
> > Is `endOfInput` in your proposal same as `BoundedOneInput.endInput` ?
>
> Yes it's the same. Sorry for a typo, somehow I was convinced its
> `endOfInput` not simple `endInput` :)
>
> Piotrek
>
> czw., 4 mar 2021 o 11:09 Kezhu Wang <ke...@gmail.com> napisał(a):
>
>
>

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi Piotr,

Very thanks for the suggestions and thoughts!

> Is this a problem? Pipeline would be empty, so EndOfPartitionEvent would be traveling very quickly.

No, this is not a problem, sorry I have some wrong thoughts here, initially in fact I'm thinking on this issue raised by 
@kezhu:

> Besides this, will FLIP-147 eventually need some ways to decide whether an operator need final checkpoint
 @Yun @Guowei ?  @Arvid mentions this in earlier mail.

For this issue itself, I'm still lean towards we might still need it, for example, suppose we have a job that 
do not need to commit anything on finished, then it do not need to wait for checkpoint at all for normal
finish case.

> Yes, but aren't we doing it right now anyway? `StreamSource#advanceToEndOfEventTime`?

Yes, we indeed have advancedEndOfEventTime for both legacy and new sources, sorry for the overlook.

> Is this the plan? That upon recovery we are restarting all operators, even those that have already finished? 
Certainly it's one of the possibilities.

For the first version we would tend to use this way since it is easier to implement, and we should always need 
to consider the case that tasks are started but operators are finished since there might be also tasks with part 
of operators finished. For the long run I think we could continue to optimize it via not restart the finished tasks 
at all.

> Keep in mind that those are two separate things, as I mentioned in a previous e-mail:
> > II. When should the `GlobalCommitHandle` be created? Should it be returned from `snapshotState()`, `notifyCheckpointComplete()` or somewhere else?> > III. What should be the ordering guarantee between global commit and local commit, if any? Actually the easiest to implement would be undefined, but de facto global commit happening before local commits (first invoke > `notifyCheckpointComplete()` on the `OperatorCoordinator` and either after or in parallel send `notifyCheckpointComplete()` RPCs). As far as I can tell, undefined order should work for the use cases that I'm aware of.
>
> We could create the `GlobalCommitHandle` in `StreamOperator#snapshotState()`, while we could also ensure that `notifyCheckpointComplete()` is called on the `OperatorCoordinator` AFTER all of the operators have successfully > processed `notifyCheckpointComplete()`. This would be more difficult to implement, hence I would prefer "undefined" behaviour here, but it's probably possible.

Very thanks for the further explanation, and I also totally agree with that the two are separate and we could think on them 
distinctly. Regarding the order, I would still tend to we support the ordered case, since the sinks' implementation seem to depend
on this functionality.

Best,
 Yun


------------------------------------------------------------------
From:Piotr Nowojski <pn...@apache.org>
Send Time:2021 Mar. 4 (Thu.) 22:56
To:Kezhu Wang <ke...@gmail.com>
Cc:Till Rohrmann <tr...@apache.org>; Guowei Ma <gu...@gmail.com>; dev <de...@flink.apache.org>; Yun Gao <yu...@aliyun.com>; jingsonglee0@gmail.com <ji...@gmail.com>
Subject:Re: Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi Yun and Kezhu,

> 1. We might introduce a new type of event to notify the endOfInput() though the graph first, and then A/B waits for the finalcheckpoint, then A emit EndOfPartitionEvent to exit all the tasks as now.

As I mentioned in one of the PRs, I would opt for this solution.

>  if we go towards 1, the tasks would still need to exit from the source, and if we go towards 2/3, we could be able to allow thesetasks to finish first.

Is this a problem? Pipeline would be empty, so EndOfPartitionEvent would be traveling very quickly.

> should we also need to do it for normal exit 

Yes, but aren't we doing it right now anyway? `StreamSource#advanceToEndOfEventTime`? 

> If so, since now for recovery after some tasks finished we would first start all the tasks and stop the finished tasks directly

Is this the plan? That upon recovery we are restarting all operators, even those that have already finished? Certainly it's one of the possibilities.

> For example, the global committer handler might be write meta store for FileSystem/Hive sinks, and these should happen after all the pending
> files are renamed to the final ones, otherwise the downstream jobs might miss some files if they relies on the meta store to identify ready partitions.
> Thus we would have to emit the global-committer-handler after notifyCheckpointComplete. But since we could be able to know the list of files
> to rename in snapshotState(), we could create the global-committer-handler and store them there.

Keep in mind that those are two separate things, as I mentioned in a previous e-mail:
> II. When should the `GlobalCommitHandle` be created? Should it be returned from `snapshotState()`, `notifyCheckpointComplete()` or somewhere else?> III. What should be the ordering guarantee between global commit and local commit, if any? Actually the easiest to implement would be undefined, but de facto global commit happening before local commits (first invoke `notifyCheckpointComplete()` on the `OperatorCoordinator` and either after or in parallel send `notifyCheckpointComplete()` RPCs). As far as I can tell, undefined order should work for the use cases that I'm aware of.

We could create the `GlobalCommitHandle` in `StreamOperator#snapshotState()`, while we could also ensure that `notifyCheckpointComplete()` is called on the `OperatorCoordinator` AFTER all of the operators have successfully processed `notifyCheckpointComplete()`. This would be more difficult to implement, hence I would prefer "undefined" behaviour here, but it's probably possible.

Kezhu:

> Is `endOfInput` in your proposal same as `BoundedOneInput.endInput` ?

Yes it's the same. Sorry for a typo, somehow I was convinced its `endOfInput` not simple `endInput` :)

Piotrek
czw., 4 mar 2021 o 11:09 Kezhu Wang <ke...@gmail.com> napisał(a):


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

Posted by Kezhu Wang <ke...@gmail.com>.
Hi Piotrek,

For “end-flushing”, I presented no new candidates. I uses “end-flushing” to
avoid naming issues. It should be “close” in allowing checkpoint after
“close” while “endOfInput” in your approach. What I tried to express is we
can do some backporting work to mitigate possible breaking changes in your
approach.

To confirm to myself that whether I understand your approach correctly, I
want to confirm somethings to avoid misleading from my side:
* Is `endOfInput` in your proposal same as `BoundedOneInput.endInput` ?

I (also ?) think there are overlaps between `BoundedOneInput.endInput` and
`close`. If yes, I like your proposal to squash them to one and deprecate
`BoundedOneInput.endInput`.

> Also, `global-commit-handles` will need to be part of the checkpoint.

I had to admit that I overlooked this, sorry for this. In my previous
suggestion, I assumed `global-commit-handles` is not part of checkpoint.
Checkpoint will complete and persist as before. The changed parts are after
checkpoint considered completed.


Best,
Kezhu Wang


On March 4, 2021 at 17:15:53, Piotr Nowojski (pnowojski@apache.org) wrote:

Hi Kezhu,

What do you mean by “end-flushing”? I was suggesting to just keep
`endOfInput()` and `dispose()`. Are you suggesting to have a one
`endFlushing()` method, that is called after quiescing timers/mailbox, but
before final checkpoint and `dispose()`? Are we sure we really need this
extra call? Note. If we don't need it at the moment, we could always
introduce it in the future, while if we don't and won't need it, why
complicate the API?

About the idea of returning the global-commit-handle from
`notifyCheckpointComplete()` call. Note it will be more difficult to
implement, as `CheckpointCoordinator` will need to have one extra stage of
waiting for some actions to complete. Implementation will probably be
easier if we return the global-commit-handle from `snapshotState()` call.

Also, `global-commit-handles` will need to be part of the checkpoint. They
will need to be restored/recovered in case of failure. Because of that it
might be actually impossible to implement those handles as returned from
`notifyCheckpointComplete()`. In this solution we would be in a precarious
position if the main checkpoint succeeded, CheckpointCoordinator would
start issuing `notifyCheckpointComplete()`, but persisting of the handles
would fail/keep failing. How would we recover from such a situation? We can
not recover to a previous checkpoint (`notifyCheckpointComplete()` were
already issued), but at the same time the current checkpoint is not fully
completed (global-commit-handles can not be checkpointed).

Best,
Piotrek



czw., 4 mar 2021 o 06:33 Kezhu Wang <ke...@gmail.com> napisał(a):

> Hi all,
>
> Glad to see convergence here and FLINK-21133:
> 1. We all prefer single final checkpoint for task not individual
> checkpoints for each operators.
> 2. To above goal, if we have to breaking something, we will.
> 3. Don’t allow recording emitting in `notifyCheckpointComplete`.
>
> For “end-flushing”, I think both approaches should function in reality,
> but we also have options/responsibilities to mitigate effect of breaking
> changes:
> A. Allowing checkpoint after “close”. Introduce config option to forbid
> this during migrating releases.
> B. Renaming “close” to “other-end-flushing-method”. We can backport that
> newly introducing “end-flushing”(as empty default method) to earlier
> releases in following patch releases. The backporting “end-flushing” will
> be called just before “close” in future patch releases. We could call
> “close” just before “dispose" in future releases and `final` it in
> `AbstractStreamOperator` when ready(to break user side code).
>
> If breaking change for this “end-flushing” in inevitable, I am kind of
> prefer renaming and backport approach. It is a chance for us to rethink the
> whole thing and discard misleading “close”(currently it is mixed/misused
> with “end-flushing” and “cleanup-resource” though javadoc claims only
> “end-flushing, this could also be considered as a bug though).
>
> Besides this, will FLIP-147 eventually need some ways to decide whether an
> operator need final checkpoint @Yun @Guowei ?  @Arvid mentions this in
> earlier mail.
>
>
> For the two phase commit, @Piotrek I like your idea. I think that
> “commit-handle” could be return to checkpoint-coordinator through
> `notifyCheckpointComplete`. This way that “commit-handle” might be reused
> by operator-coordinator’s `notifyCheckpointComplete`. Suppose following
> changes:
>
> 1. `CompletableFuture<SerializedValue<?>> notifyCheckpointCompleteAsync()`
> in operator.
> 2. `CompletableFuture<Void> notifyCheckpointCompleteAsync(Map<Integer,
> CompletableFuture<SerializedValue>> subtasks)` in operator coordinator.
>
> These changes need support from:
> * Checkpoint coordinator level to bridge operator and coordinator through
> task
> * Operator level to compat existing `notifyCheckpointComplete`
>
> The checkpoint procedure will looks like:
> 1. Trigger checkpoint for operator coordinator.
> 2. If above succeeds, trigger tasks checkpoint. Abort otherwise.
> 3. If all above succeeds, complete current checkpoint. Abort otherwise.
> 4. If job fails after, restore from above “completed” checkpoint.
> 5. Notify checkpoint completion to tasks.
> 6. Notify checkpoint completion to coordinators.
> 7. Wait step#5 and step#6 to succeed. Now it is real completed. Either
> this succeed or job failed in meantime ? May be other concurrent conditions.
>
> With these changes, migration FLIP-143 sink to operator coordinator should
> be easy.
>
> It will definitely complicate currently complex checkpoint coordinator as
> @Till mentioned in FLINK-21133.
>
>
> Best,
> Kezhu Wang
>
> On March 3, 2021 at 01:09:50, Piotr Nowojski (pnowojski@apache.org) wrote:
>
> Hi,
>
> Thanks for reminding me. I think FLIP-147 will have to deal in one way or
> another with the (re?)emitting MAX_WATERMARK. There is a pre-existing
> issue
> that watermarks are not checkpointed/stored on state, and there was/is now
> clear answer how we should handle this as far as I remember. One
> problematic case are two/multiple input tasks or UnionInputGate, where
> combined watermark is the min of all inputs (held in memory). The problem
> so far is a bit benign, as after recovery we are losing the combined
> watermark value, but it's being slowly/lazily restored, as new watermarks
> are sent from the sources. With finished sources that won't be a case.
>
> I've spent more time thinking about the two phase commit issue, and I
> agree
> it would be best to have a single final checkpoint for the operators. The
> more I think about it, the more I'm leaning toward the
> "OperatorCoordinator" approach. Yes, it would need the extra complexity of
> running user code on the JobManager, but that's already at least partially
> done (FLIP-27). On the other hand it would allow us to keep the runtime
> code cleaner and simpler I think. It's still an invasive change.
>
> 1. We forbid emitting records from
> `notifyCheckpointComplete`/`snapshotState` calls, or at the very least
> from
> the final calls.
> 2. We need to clarify/clean up contracts for both operators AND user
> functions when it comes to `endOfInput()`, `close()` and `dispose()`
> methods. For example:
> a) we keep `endOfInput()` on the operators level and add an equivalent of
> it to the user functions
> b) we move the "flushing buffered records" part of the `close()` contract
> to `endOfInput()`
> c) we remove `close()` methods altogether (IMO it's better to have an
> explicit false conflict, rather than a silent real one)
> d) we keep `dispose()` on the operator level for release resources, and
> we add such method to the user functions
> 3. We change (or add another version) a return type of either `void
> snapshotState(...)` or `void notifyCheckpointComplete(...)` calls, so that
> they will return some kind of `GlobalCommitHandle`.
>
> Especially this point 2. is intrusive. Now, putting all of that together.
> When a task receives a logical "end of input" signal (current
> `EndOfPartitionEvent`), it begins shutting down procedure (let's assume we
> create `GlobalCommitHandle` in `snapshotState`).
>
> 0. n = 1
> 1. endOfInput on the n'th operator
> 2. quisec timers and mailbox executor for n'th operator
> // after this point n'th operator is forbidden to emit any new records
> 3. n += 1, and go to 1., until we handle all of the operators
> 4. wait for final checkpoint, issue `snapshotState` and collect
> `GlobalCommitHandle`s and send them to the JM
> 5. JM/CheckpointCoordinator collects all `GlobalCommitHandles`. They are
> an
> integral part of the checkpoint. They could be added to the respective
> `OperatorCoordinator`s via `handleEventFromOperator` as an
> `OperatorEvent`.
> 6. When checkpoint is considered completed, `notifyCheckpointComplete` are
> being issued, both on the operators, and `OperatorCoordinator` - local
> commit and global commits would be happening at the same time, or even
> global commit code would be executed before local final commit in this
> version.
> 7. Only now can we finally dispose of all operators in the task.
>
> If there is a failure, especially before we manage to perform all
> `notifyCheckpointComplete` calls (for example on the
> `OperatorCoordinator`), we would need to recover from last "completed"
> checkpoint, recover attached `GlobalCommitHandle`s , and re-commit them.
> It
> also means we need to recover already finished operators, that will never
> process any records, just to issue the final `notifyCheckpointComplete`
> call and make sure they commit their external side effects.
>
> There are at least a couple of questions about this general idea:
> I. Naming of the methods (I suggested to drop `close()` and keep in the
> operators and introduce in the functions `endOfInput()` and `dispose()`
> II. When should the `GlobalCommitHandle` be created? Should it be returned
> from `snapshotState()`, `notifyCheckpointComplete()` or somewhere else?
> III. What should be the ordering guarantee between global commit and local
> commit, if any? Actually the easiest to implement would be undefined, but
> de facto global commit happening before local commits (first invoke
> `notifyCheckpointComplete()` on the `OperatorCoordinator` and either after
> or in parallel send `notifyCheckpointComplete()` RPCs). As far as I can
> tell, undefined order should work for the use cases that I'm aware of.
> IV. Can we reuse `OperatorCoordinator` abstraction for this purpose?
> Currently it's being used to induce checkpoints with FLIP-27 sources as
> far
> as I understand, here we would be using it for different purposes.
>
> Best,
> Piotrek
>
> pon., 1 mar 2021 o 07:30 Yun Gao <yu...@aliyun.com> napisał(a):
>
> > (Sorry that I repeat this mail since the last one is not added into the
> > same mail list thread,
> > very sorry for the inconvenience)
> >
> > Hi all,
> >
> > Very thanks for all the deep thoughts!
> >
> > > How to implement the stop-with-savepoint --drain/terminate command
> with
> > > this model: One idea could be to tell the sources that they should
> stop
> > > reading. This should trigger the EndOfPartitionEvent to be sent
> > > downstream.
> > > This will transition all operators into the TERMINATING state.
> > >
> > > Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
> > > returned. To achieve above, possible works should be required:
> > > * Promote `EndOfPartitionEvent` from `Task` to `StreamTask`. This may
> > > have some interferences with BatchTask or network io stack.
> > > * Or introducing stream task level `EndOfUserRecordsEvent`(from
> PR#14831
> > > @Yun @Piotr)
> > > * Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.
> >
> > I also have similar concern with Kezhu for the issue whether do we need
> to
> > introduce a new message
> > to notify the operators to endOfInput/close ? The main concerns of
> reusing
> > that EndOfPartitionEvent
> > is that
> > 1. The EndOfPartitionEvent is currently emitted in Task instead of
> > StreamTask, we would need some
> > refactors here.
> > 2. Currently the InputGate/InputChannel would be released after the
> > downstream tasks have received
> > EndOfPartitionEvent from all the input channels, this would makes the
> > following checkpoint unable to
> > perform since we could not emit barriers to downstream tasks ?
> >
> > Regarding the MAX_WATERMARK, I still not fully understand the issue
> since
> > it seems to me
> > that now Flink won't snapshot the watermark now? If the job failover,
> the
> > window operator
> > would reload all the pending windows before flushed by MAX_WATERMARK and
> > when the
> > job finish again, it would re-emit the MAX_WATERMARK?
> >
> > Best,
> > Yun
> >
> >
> > ------------------------------------------------------------------
> > From:Kezhu Wang <ke...@gmail.com>
> > Send Time:2021 Mar. 1 (Mon.) 01:26
> > To:Till Rohrmann <tr...@apache.org>
> > Cc:Piotr Nowojski <pi...@gmail.com>; Guowei Ma <
> > guowei.mgw@gmail.com>; dev <de...@flink.apache.org>; Yun Gao <
> > yungao.gy@aliyun.com>; jingsonglee0@gmail.com <ji...@gmail.com>
> > Subject:Re: Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > Finished
> >
> > In “stop-with-savepoint —drain”, MAX_WATERMARK is not an issue. For
> normal
> > finishing task, not allowing unaligned checkpoint does not solve the
> > problem as MAX_WATERMARK could be persisted in downstream task. When
> > scenario @Piotr depicted occurs, downstream(or further downstream)
> window
> > operator will count all inputs as late.
> >
> > > If we ensure that the MAX_WATERMARK is only persisted in state if a
> > recovery will trigger immediately the shut down of this operator, then
> it
> > shouldn't be an issue.
> >
> > You are right in case the assumption holds, I have same thought as you
> > before. But I am kind of worry about whether it is too prefect to be
> > fragile. This requires strong guarantee from implementation that a
> recovery
> > from TERMINATING stage should go directly to that stage.
> >
> > > I think the MAX_WATERMARK event should be sent either just before or
> with
> > the EndOfPartitionEvent.
> >
> > I prefer “with the EndOfPartitionEvent”. EndOfPartitionEvent itself
> already
> > carry what ending MAX_WATERMARK try to express. May be we can reuse it ?
> > @Piotr
> >
> > A preview work[1] from @Yun in PR#14831 explains EndOfPartitionEvent as
> > checkpoint barrier if there are pending checkpoints.
> >
> >
> > [1]:
> >
> >
> https://github.com/gaoyunhaii/flink/commit/1aaa80fb0afad13a314b06783c61c01b9414f91b#diff-d4568b34060bc589cd1354bd125c35aca993dd0c9fe9e4460dfed73501116459R177
> >
> >
> > Best,
> > Kezhu Wang
> >
> > On February 28, 2021 at 21:23:31, Till Rohrmann (trohrmann@apache.org)
> > wrote:
> >
> > I think you are right with the problem of endOfInput. endOfInput should
> not
> > be used to commit final results. In fact if this termination fails then
> we
> > might end up in a different outcome of the job which is equally valid as
> > the one before the failure.
> >
> > Concerning unaligned checkpoints, I think they don't play well together
> > with draining a streaming pipeline. The problem is that in the draining
> > case you want to process all records which are still in flight but
> > unaligned checkpoints don't guarantee this as they can jump in flight
> > records.
> >
> > I think the MAX_WATERMARK event should be sent either just before or
> with
> > the EndOfPartitionEvent. If we ensure that the MAX_WATERMARK is only
> > persisted in state if a recovery will trigger immediately the shut down
> of
> > this operator, then it shouldn't be an issue.
> >
> > Cheers,
> > Till
> >
> > On Sun, Feb 28, 2021 at 9:06 AM Kezhu Wang <ke...@gmail.com> wrote:
> >
> > > Hi Till,
> > >
> > > Just for bookkeeping, some observations from current implementation.
> > >
> > > > With this model, the final checkpoint is quite simple because it is
> > > ingrained in the lifecycle of an operator. Differently said an
> operator
> > > will only terminate after it has committed its side effects and seen
> the
> > > notifyCheckpointComplete message (if it is stateful).
> > >
> >
> > > Currently, we could not mark this operator(or subtask) as terminated
> since
> > > result of `notifyCheckpointComplete`(possible side effect committing)
> is
> > > not taken into account of the belonging checkpoint. The job has to run
> to
> > > next safe point(finished or next checkpoint success) to be marked as
> > > “terminated”.
> > >
> > > > How to implement the stop-with-savepoint --drain/terminate command
> with
> > > this model: One idea could be to tell the sources that they should
> stop
> > > reading. This should trigger the EndOfPartitionEvent to be sent
> > > downstream.
> > > This will transition all operators into the TERMINATING state.
> > >
> > > Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
> > > returned. To achieve above, possible works should be required:
> > > * Promote `EndOfPartitionEvent` from `Task` to `StreamTask`. This may
> > > have some interferences with BatchTask or network io stack.
> > > * Or introducing stream task level `EndOfUserRecordsEvent`(from
> PR#14831
> > > @Yun @Piotr)
> > > * Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.
> > >
> > > Besides this, I would like to quote some discussion from FLINK-21467
> > > between @Piotr and me:
> > >
> > > From @Piotr
> > > > Note, that it's not only that endOfInput can be called multiple
> times.
> >
> > > There is a very remote possibility that the following scenario will
> happen:
> > > 1. checkpoint is taken (successfully)
> > > 2. sources are finishing
> > > 3. endOfInput is issued
> > > 4. job fails
> > > 5. job restarts to checkpoint 1.
> > > 6. after failover, because of some non deterministic logic in the
> source,
> > > sources are not finishing
> > >
> > > From me
> > > > But I think there is little work Flink can do to cope with this kind
> of
> > > issues. The checkpoint could be a savepoint triggered from user side
> and
> > > the "non deterministic logic" could be a change from user(eg. changing
> of
> > > stoppingOffsets in KafkaSource).
> > >
> >
> > > > I think the "non deterministic logic" could cause trouble in
> combination
> > > with unaligned checkpoint and downstream window operator. Unaligned
> > > checkpoint will persist "MAX_WATERMARK" in state, after restarting,
> > > "MAX_WATERMARK" will cause downstream window operator ignores all
> future
> > > inputs.
> > >
> > > FLIP-147 demands no new records from end-of-stream-flushing, but
> source
> > > will emit “MAX_WATERMARK” on ending. Previously, I thought it is not a
> >
> > > valid issue, but turn out that it could cause trouble under scenario
> listed
> > > by @Piotr if I am not wrong.
> > >
> > >
> > > PR#14831: https://github.com/apache/flink/pull/14831
> > > FLINK-21467: https://issues.apache.org/jira/browse/FLINK-21467
> > >
> > >
> > > Best,
> > > Kezhu Wang
> > >
> > > On February 27, 2021 at 18:12:20, Till Rohrmann (trohrmann@apache.org)
>
> > > wrote:
> > >
> > > Thanks for all your thoughts. I think we should further think through
> >
> > > whether to allow checkpoints after an operator has emitted all its
> records
> > > (e.g. after close is called currently) or not. I think by doing this
> we
> > > would nicely decouple the checkpoint taking from the operator
> lifecycle
> > > and
> > > wouldn't need special checkpoints/savepoints for the final checkpoint
> and
> > > stop-with-savepoint --drain. Let me try to explain this a bit more
> > > detailed.
> > >
> > > If we say an operator has the rough lifecycle RUNNING => TERMINATING
> =>
> > > TERMINATED where we go from RUNNING into TERMINATING after we have
> seen
> > > the
> > > EndOfPartitionEvent and flushed all our records. The operator goes
> from
> > > TERMINATING => TERMINATED if it has persisted all its possible side
> > > effects. Throughout all states, it is possible to trigger a
> checkpoint. A
> > > stateless operator will immediately go from TERMINATING to TERMINATED
> > > whereas a stateful operator would wait for another checkpoint to be
> > > triggered and successfully completed (notifyCheckpointComplete).
> > >
> > > With this model, the final checkpoint is quite simple because it is
> > > ingrained in the lifecycle of an operator. Differently said an
> operator
> > > will only terminate after it has committed its side effects and seen
> the
> >
> > > notifyCheckpointComplete message (if it is stateful). Here it is
> important
> > > to note that in the streaming case, different bounded operators can
> > > terminate at different times. They don't have to terminate all with
> the
> > > same checkpoint.
> > >
> > > How to implement the stop-with-savepoint --drain/terminate command
> with
> > > this model: One idea could be to tell the sources that they should
> stop
> > > reading. This should trigger the EndOfPartitionEvent to be sent
> > > downstream.
> >
> > > This will transition all operators into the TERMINATING state. Next
> the JM
> > > can trigger a checkpoint to shut the operators down and not to wait
> for
> > > the
> > > next automatic checkpoint trigger event.
> > >
> > > By allowing checkpoints throughout the entire lifecycle of an operator
> we
> > > disallow sending records from notifyCheckpointComplete because this
> > > message
> > > will also be sent in the state TERMINATING where an operator has
> already
> > > produced all of its records.
> > >
> > > What do you think? Maybe this model overlooks some important cases.
> One
> > > downside is that we will break the operator API with changing the
> > > lifecycle
> > > of an operator.
> > >
> > > Cheers,
> > > Till
> > >
> > >
> > >
> > > On Fri, Feb 26, 2021 at 7:29 AM Kezhu Wang <ke...@gmail.com> wrote:
> > >
> > > > Hi Yun,
> > > >
> > > > The termination phase I drafted depends on a de-fact that
> >
> > > > `notifyCheckpointComplete` is ignored currently after `close`. Thus,
> any
> > > > external materialization inside `close`(including `endInput`) or
> > > > `notifyCheckpointComplete` is either not guaranteed or not
> committed.
> > > >
> > > > I tried to emphasize no-breaking changes and post-pone migration in
> > > later
> > > > releases for this termination phase. But now, I am kind of worry
> about
> > > > whether it will cause long-term maintenance hell.
> > > >
> > > > Personally, I think allowing checkpoint after `close`(@Till proposed
> > > this
> > > > in FLINK-21133) could be the minimal change. But there are concerns:
> > > > * It will break some existing code possibly in a silent way.(@Piotr
> > > pointed
> > > > this already in FLINK-21133)
> > > > * I think the name `close` is kind of misleading. (@Piotr suggested
> > > > renaming this to `finish` to not break code silently in FLINK-21133)
> > > >
> > > > > stop-with-savepoint --drain would wait for a specific savepoint
> > > >
> > > > For stop-with-savepoint, the checkpoint barrier is already created
> or
> > > > received there.
> > > >
> > > >
> > > > Best,
> > > > Kezhu Wang
> > > >
> > > > On February 25, 2021 at 21:02:30, Yun Gao (yungao.gy@aliyun.com
> > ) wrote:
> > > >
> > > > Hi all,
> > > >
> > > > Very thanks for the discussions!
> > > >
> > > >
> > > >
> > > > A. Regarding how to avoid emitting records in
> notifyCheckpointComplete:
> > > >
> > > > Currently the structure of a new sink is writer -> committer ->
> global
> > > > committer and the paralellism of global committer
> > > > must be one. By design it would be used in several cases:
> > > > 1. writer -> committer: for normal sinks that write pending data in
> > > writer
> > > > and commit pending data in committer.
> > > > 2. writer -> global committer: for sinks require the committer's
> > > > parallelism be 1, like iceberg sink which
> > > > relies on optimistic lock to commit data hopes to reduce the
> conflicts.
> > > > 3. writer -> committer -> global committer: for sinks that also need
> to
> > > > write meta data (like _SUCCESS file
> > > > or add record in hive metastore) in global committer.
> > > >
> > > > The third case would cause the cascade commit problem. To overcome
> this
> > > > problem, we would like to
> > > > 1. Previously the global committer also support writing metadata
> with
> > > 2pc.
> > > > Now we disable this mode
> > > > and always rely on the property that writing metadata is repeatable.
> > > With
> > > > this limitation there should no
> > > > need of cascade commit, but the committer would still emit records
> in
> > > > notifyCheckpointComplete.
> > > > 2. We further move global committer in the case 3 to the operator
> > > > coordinator. Then the committer would
> > > > not need to emit records, but communicates with its operator
> > > coordinator.
> > > >
> > > > One core issue for using the OperatorCoordinator is how to keep the
> > > > communication between the operator
> > > > and the OperatorCoordinator exactly-once. Since the message is
> always
> > > from
> > > > the oeprator to the OperatorCoordinator
> > > > in this case, we would only need to bookkeep the message sent
> between
> > > the
> > > > OperatorCoordinator takes snapshot and
> > > > the Operator takes snapshot in the state of the Operator.
> > > >
> > > > On how to achieve the change in detail we would still need some
> think,
> > > it
> > > > currently seems we would have to had
> > > > some modification to the current new sink api.
> > > >
> > > >
> > > >
> > > > B. Regarding the stop-with-savepoint --drain
> > > >
> > > > Very thanks @Piotr for the further explanation and now I realize I
> have
> > > > understand wrongly for the semantics of
> > > > stop-with-savepoint --drain. Now I think that the problem should be
> we
> > > > should also include the records produced in
> > > > `endOfInput()` and `close()` also in the last savepoint, am I
> correct?
> > > If
> > > > so, it seems we still have some undetermined options for
> > > > the lifecycle of the operator, like in Kezhu's proposal the close()
> > > happens
> > > > at last, but it seems close() might also emit records (
> > > > so now the operator are closed with op1's close() -> op2's
> endOfInput()
> > > ->
> > > > op2's close() -> op3's endOfinput -> ...) ?
> > > >
> > > > And on the other side, as Kezhu has also proposed, perhapse we might
> > > have
> > > > the stop-with-savepoint --drain and normal exit in the same process,
> > > > but have slightly difference in that stop-with-savepoint --drain
> would
> > > wait
> > > > for a specific savepoint and in normal exit, the operator
> > > > might wait for arbitrary checkpoint. If we could achieve not
> emitting
> > > > records in notifyCheckpointComplete, stop-with-savepoint --drain
> could
> > > > be done with one savepoint, and for the normal exit, the operator
> would
> > > not
> > > > need to wait for other slow operators to exit.
> > > >
> > > > Best,
> > > > Yun
> > > >
> > > >
> > > >
> > > > ------------------Original Mail ------------------
> > > > *Sender:*Kezhu Wang <ke...@gmail.com>
> > > > *Send Date:*Thu Feb 25 15:11:53 2021
> > > > *Recipients:*Flink Dev <de...@flink.apache.org>, Piotr Nowojski <
> > > > piotr.nowojski@gmail.com>
> > > > *CC:*Guowei Ma <gu...@gmail.com>, jingsonglee0@gmail.com <
> > > > jingsonglee0@gmail.com>
> > > > *Subject:*Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After
> Tasks
> > > > Finished
> > > >
> > > > > Hi all, thanks for driving this and especially Piotr for re-active
> > > this
> > > > >
> > > > > thread.
> > > > >
> > > > >
> > > > >
> > > > > First, for `notifyCheckpointComplete`, I have strong preference
> > > towards
> > > > >
> > > > > "shut down the dataflow
> > > > >
> > > > > pipeline with one checkpoint in total", so I tend to option
> dropping
> > > > "send
> > > > >
> > > > > records" from
> > > > >
> > > > > `notifyCheckpointComplete` for next level committing in pipeline,
> if
> > > we
> > > > >
> > > > > ever support it. Without this,
> > > > >
> > > > > we are unable to stop a pipeline manually with all results
> > > materialized.
> > > > >
> > > > >
> > > > >
> > > > > Second, for shutdown unification of `stop-with-savepoint --drain`
> and
> > > > >
> > > > > FLIP-147, I draft following
> > > > >
> > > > > phase based on emerging proposals(mainly by Piotr and Till) in
> this
> > > > thread
> > > > >
> > > > > and FLINK-21133.
> > > > >
> > > > >
> > > > >
> > > > > ```java
> > > > >
> > > > > // StreamTask termination phase
> > > > >
> > > > > finish()(Call StreamOperator.finish in chaining order)
> > > > >
> >
> > > > > advanceToEndOfEventTime()(nop if for no-source inputs, this could
> also
> > > be
> > > > >
> > > > > done in input processor or finish ?)
> > > > >
> > > > > if (there-is-a-pending-terminate-savepoint) {
> > > > >
> > > > > triggerBarrierForDownStream();
> > > > >
> > > > > waitCheckpointComplete();
> > > > >
> > > > > } else if (require-two-phase-commit-in-shutdown) {
> > > > >
> > > > > waitFinalCheckpoint();
> > > > >
> > > > > waitCheckpointComplete();
> > > > >
> > > > > }
> > > > >
> > > > > // TODO: Ignore/Cancel all checkpoints from this point(or hit to
> > > > >
> > > > > coordinator in final checkpoint response ?).
> > > > >
> > > > > close();
> > > > >
> > > > > ```
> > > > >
> > > > >
> > > > >
> > > > > Branches in `if` could reside in different code paths, but the
> effect
> > > > >
> > > > > should be similar to above.
> > > > >
> > > > > The divergence reflects fact that "stop-with-savepoint --drain" is
> > > > >
> > > > > triggered already while we need
> > > > >
> > > > > to wait for final checkpoint in natural input exhausting.
> > > > >
> > > > >
> > > > >
> > > > > With carefully chosen default functions, we will not break
> existing
> > > > >
> > > > > interfaces.
> > > > >
> > > > > * `StreamOperator.finish`
> > > > >
> > > > > * `RichFunction.finish`
> > > > >
> > > > >
> > > > >
> > > > > For existing codes:
> > > > >
> > > > > 1. "stop-with-savepoint" with no migration, it should behaves as
> > > before.
> > > > >
> > > > > 2. "require-two-phase-commit-in-shutdown" evaluates to false and
> no
> > > > >
> > > > > migration, it is same as before.
> > > > >
> > > > > 3. "require-two-phase-commit-in-shutdown" evaluates to true and no
> > > > >
> > > > > migration, records in `close` will
> > > > >
> > > > > lose.
> > > > >
> > > > >
> > > > >
> > > > > For case#3:
> > > > >
> > > > > A. No external materialization in `StreamOperator.close`: Nothing
> to
> > > > lose.
> > > > >
> > > > > B. External materialization only in `StreamOperator.close`: I
> don't
> > > think
> > > > >
> > > > > Flink ever claimed that there is
> > > > >
> > > > > any guarantee for this situation.
> > > > >
> > > > > C. External materialization in `notifyCheckpointComplete` from
> > > > >
> > > > > `StreamOperator.close`: But due to fact that
> > > > >
> > > > > `notifyCheckpointComplete` was ignored after operator
> > > > >
> > > > > closed(FLINK-16383), so there will be no external
> > > > >
> > > > > materialization to lose.
> > > > >
> > > > >
> > > > >
> > > > > Then, we could recommend users to migrate possible
> > > > “end-of-stream-flushing”
> > > > >
> > > > > from “close" to “finish”.
> > > > >
> > > > >
> > > > >
> > > > > Best,
> > > > >
> > > > > Kezhu Wang
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On February 24, 2021 at 23:47:36, Piotr Nowojski (
> > > > piotr.nowojski@gmail.com
> > > > > )
> > > > >
> > > > > wrote:
> > > > >
> > > > >
> > > > >
> > > > > Thanks for the reponses Guowei and Yun,
> > > > >
> > > > >
> > > > >
> > > > > Could you elaborate more/remind me, what does it mean to replace
> > > emitting
> > > > >
> >
> > > > > results from the `notifyCheckpointComplete` with
> `OperatorCoordinator`
> > > > >
> > > > > approach?
> > > > >
> > > > >
> > > > >
> > > > > About the discussion in FLINK-21133 and how it relates to
> FLIP-147.
> > > You
> > > > are
> > > > >
> >
> > > > > right Yun gao, that in case of `stop-with-savepoint --drain` the
> whole
> > > > job
> > > > >
> > > > > finishes, while in FLIP-147 some of the sources can work for
> > > arbitrarily
> > > > >
> >
> > > > > long time after others have already finished. However from the
> runtime
> > > > >
> > > > > perspective, especially how to close a Task and it's operators,
> there
> > > is
> > > > no
> > > > >
> > > > > difference between those two cases. In both cases, we need to end
> > > input,
> > > > >
> >
> > > > > shut down processing time timers and commit external side effects
> (two
> > > > >
> > > > > phase commit support) before exiting the task. That's how the
> > > discussion
> > > > >
> > > > > about the behaviour of "stop-with-savepoint" was connected with
> > > FLIP-147.
> > > > >
> > > > >
> > > > >
> > > > > Currently on master, "stop-with-savepoint --drain" drains/flushes
> > > > buffered
> > > > >
> >
> > > > > records and deals correctly with timers, but all of that happens
> AFTER
> > > > >
> > > > > savepoint was completed. So any records flushed from the operators
> > > during
> > > > >
> > > > > endOfInput/close/shutting down processing timers are never
> committed
> > > to
> > > > >
> > > > > external systems. This is exactly the same problem as the "two
> phase
> > > > >
> >
> > > > > commit" problem of FLIP-147, that should have the same solution
> and it
> > > > >
> >
> > > > > should be solved at the same time. For example if we go with the
> *very
> > > > >
> > > > > undesirable* "one closed operator per one completed
> > > > checkpoint/savepoint",
> > > > >
> > > > > in both cases CheckpointCoordinator, Scheduler and Task would need
> to
> > > > keep
> > > > >
> > > > > the task alive and keep triggering checkpoints for that task,
> until
> > > all
> > > > >
> > > > > operators in the operator chain are closed (one closed operator
> per
> > > one
> > > > >
> > > > > completed checkpoint).
> > > > >
> > > > >
> > > > >
> > > > > Piotrek
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > śr., 24 lut 2021 o 11:30 Yun Gao napisał(a):
> > > > >
> > > > >
> > > > >
> > > > > > Hi Till, Guowei,
> > > > >
> > > > > >
> > > > >
> > > > > > Very thanks for initiating the disucssion and the deep thoughts!
> > > > >
> > > > > >
> > > > >
> >
> > > > > > For the notifyCheckpointComplete, I also agree we could try to
> avoid
> > > > >
> > > > > > emitting new records in notifyCheckpointComplete via using
> > > > >
> > > > > > OperatorCoordinator
> > > > >
> > > > > > for new sink API. Besides, the hive sink might also need some
> > > > >
> > > > > modification
> > > > >
> > > > > > for it also emits records in notifyCheckpointComplete.
> > > > >
> > > > > >
> > > > >
> >
> > > > > > For unifying the process of stopping with savepoint and finished
> due
> > > to
> > > > >
> > > > > > all records
> > > > >
> > > > > > are processed, I also agree with that unifying would always be
> > > better
> > > > if
> > > > >
> > > > > > we could achieve,
> > > > >
> > > > > > but I'm still not fully catch up with the implementation: Based
> on
> > > the
> > > > >
> > > > > > discussion in FLINK-21133,
> > > > >
> >
> > > > > > my understanding is that for stopping with savepoint, now we
> want to
> > > > >
> > > > > first
> > > > >
> > > > > > stop the source, then we
> > > > >
> > > > > > trigger a savepoint, and after the source received
> > > > >
> > > > > > notifyCheckpointComplete, the source would
> > > > >
> >
> > > > > > start emitting EndOfPartitionEvent to finish the job, am I
> correct ?
> > > > >
> > > > > >
> > > > >
> > > > > > For normal finish, a difference to me might be if we have
> multiple
> > > > >
> > > > > > sources, we could not guarantee
> > > > >
> > > > > > when the sources are to finish. We might have one source run one
> 1
> > > > minute
> > > > >
> > > > > > and another one run for
> > > > >
> >
> > > > > > 1 hour. To unify with the process with stop with savepoint, we
> might
> > > > need
> > > > >
> > > > > > to hold the fast source until
> > > > >
> > > > > > all the sources are finished? An coordinator would be introduced
> to
> > > > count
> > > > >
> > > > > > the number of sources
> > > > >
> >
> > > > > > runing and trigger the final savepoint / checkpoint. For the
> extreme
> > > > >
> > > > > > cases, if we have both bounded and
> > > > >
> > > > > > unbounded sources, we might only count how much bounded source
> are
> > > > >
> > > > > > remaining ? And if all the bounded
> > > > >
> > > > > > sources are finished we would trigger the special checkpoint.
> After
> > > all
> > > > >
> > > > > > the bounded part of the graph are
> > > > >
> > > > > > finished, the the remaining part could still do checkpoint and
> > > commit
> > > > >
> > > > > data
> > > > >
> > > > > > with FLIP-147.
> > > > >
> > > > > >
> > > > >
> > > > > > Best,
> > > > >
> > > > > > Yun
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > ------------------Original Mail ------------------
> > > > >
> > > > > > Sender:Guowei Ma
> > > > >
> > > > > > Send Date:Wed Feb 24 17:35:36 2021
> > > > >
> > > > > > Recipients:dev
> > > > >
> > > > > > CC:Arvid Heise
> > > > >
> > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > > > Finished
> > > > >
> > > > > > Hi, Till
> > > > >
> > > > > >
> > > > >
> > > > > > Thank you very much for your careful consideration
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > *1. Emit records in `NotifyCheckpointComplete`.*
> > > > >
> > > > > >
> > > > >
> > > > > > Sorry for making you misunderstanding because of my expression.
> I
> > > just
> > > > >
> > > > > >
> > > > >
> > > > > > want to say the current interface does not prevent users from
> doing
> > > it.
> > > > >
> > > > > >
> > > > >
> > > > > > From the perspective of the new sink api, we might not depend on
> > > > emitting
> > > > >
> > > > > >
> > > > >
> > > > > > records in `NotifyCheckpointComplete`, like using
> > > `OperatorCoordinator`
> > > > >
> > > > > >
> > > > >
> > > > > > instead.
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > *2. What does the FLIP-147 guarantee?*I think initially this
> FLIP
> > > want
> > > > to
> > > > >
> > > > > >
> > > > >
> > > > > > achieve two targets:
> > > > >
> > > > > >
> > > > >
> >
> > > > > > 1. Tasks/Operators exit correctly (as you mentioned the
> lifecycle of
> > > a
> > > > >
> > > > > >
> > > > >
> > > > > > Task/StreamTask/StreamOperator.).
> > > > >
> > > > > >
> > > > >
> > > > > > 2. Continue to trigger checkpoint after some tasks for mixed
> jobs.
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > I think the first thing is related to the discussion in
> FLINK-21133.
> > > > If I
> > > > >
> > > > > >
> > > > >
> > > > > > understand correctly, in addition to supporting the tasks /
> > > operators
> > > > to
> > > > >
> > > > > >
> > > > >
> > > > > > exit correctly, now we also want to unify the process of the
> tasks
> > > and
> > > > >
> > > > > >
> > > > >
> > > > > > operators for savepoint / finish.
> > > > >
> > > > > >
> > > > >
> > > > > > I think the second thing is orthogonal to the FLINK-21133
> because
> > > there
> > > > >
> > > > > are
> > > > >
> > > > > >
> > > > >
> > > > > > topologies that have both the bounded and unbounded input.
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > *3. How to unify the operator exit process of FLIP-147 with
> > > > >
> > > > > >
> > > > >
> > > > > > stop-with-savepoint?*
> > > > >
> > > > > >
> > > > >
> > > > > > I am not very sure about how to do it yet. But if I understand
> the
> > > > >
> > > > > >
> > > > >
> > > > > > discussion in the jira correctly it needs to introduce some
> logic
> > > into
> > > > >
> > > > > >
> > > > >
> >
> > > > > > `CheckpointCoordinator`, which responses for triggering “the
> unified
> > > > >
> > > > > >
> > > > >
> > > > > > operator exit process”. Am I correct?
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > Best,
> > > > >
> > > > > >
> > > > >
> > > > > > Guowei
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann wrote:
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > Thanks for the explanation Yun and Guowei. I have to admit
> that I
> > > do
> > > > >
> > > > > not
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > fully understand why this is strictly required but I think
> that we
> > > > are
> > > > >
> > > > > >
> > > > >
> > > > > > > touching two very important aspects which might have far
> fetching
> > > > >
> > > > > >
> > > > >
> > > > > > > consequences for how Flink works:
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > 1) Do we want to allow that multiple checkpoints are required
> to
> > > > >
> > > > > >
> > > > >
> > > > > > > materialize results?
> > > > >
> > > > > >
> > > > >
> > > > > > > 2) Do we want to allow to emit records in
> > > notifyCheckpointComplete?
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > For 1) I am not sure whether this has been discussed within
> the
> > > > >
> > > > > community
> > > > >
> > > > > >
> > > > >
> > > > > > > sufficiently. Requiring multiple checkpoints to materialize a
> > > result
> > > > >
> > > > > >
> > > > >
> > > > > > > because of multi level committers has the consequence that we
> > > > increase
> > > > >
> > > > > > the
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > latency from checkpoint interval to #levels * checkpoint
> interval.
> > > > >
> > > > > >
> > > > >
> > > > > > > Moreover, having to drain the pipeline in multiple steps,
> would
> > > break
> > > > >
> > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > stop-with-savepoint --drain because which savepoint do you
> report
> > > to
> > > > >
> > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > user?
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > For 2) allowing to send records after the final
> > > > >
> > > > > notifyCheckpointComplete
> > > > >
> > > > > >
> > > > >
> > > > > > > will effectively mean that we need to shut down a topology in
> > > > multiple
> > > > >
> > > > > >
> > > > >
> > > > > > > steps (in the worst case one operator per checkpoint). This
> would
> > > be
> > > > a
> > > > >
> > > > > >
> > > > >
> > > > > > > strong argument for not allowing this to me. The fact that
> users
> > > can
> > > > >
> > > > > send
> > > > >
> > > > > >
> > > > >
> > > > > > > records after the notifyCheckpointComplete is more by accident
> > > than
> > > > by
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > design. I think we should make this a very deliberate decision
> and
> > > in
> > > > >
> > > > > > doubt
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > I would be in favour of a more restrictive model unless there
> is a
> > > > very
> > > > >
> > > > > >
> > > > >
> > > > > > > good reason why this should be supported.
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > Taking also the discussion in FLINK-21133 [1] into account, it
> > > seems
> > > > to
> > > > >
> > > > > > me
> > > > >
> > > > > >
> > > > >
> > > > > > > that we haven't really understood what kind of guarantees we
> want
> > > to
> > > > >
> > > > > give
> > > > >
> > > > > >
> > > > >
> > > > > > > to our users and how the final checkpoint should exactly work.
> I
> > > > >
> > > > > > understand
> > > > >
> > > > > >
> > > > >
> > > > > > > that this is not included in the first scope of FLIP-147 but I
> > > think
> > > > >
> > > > > this
> > > > >
> > > > > >
> > > > >
> > > > > > > is so important that we should figure this out asap. Also
> because
> > > the
> > > > >
> > > > > > exact
> > > > >
> > > > > >
> > > > >
> > > > > > > shut down behaviour will have to be aligned with the lifecycle
> of
> > > a
> > > > >
> > > > > >
> > > > >
> > > > > > > Task/StreamTask/StreamOperator. And last but not least because
> > > other
> > > > >
> > > > > >
> > > > >
> > > > > > > features such as the new sink API start building upon a shut
> down
> > > > model
> > > > >
> > > > > >
> > > > >
> > > > > > > which has not been fully understood/agreed upon.
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > [1] https://issues.apache.org/jira/browse/FLINK-21133
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > Cheers,
> > > > >
> > > > > >
> > > > >
> > > > > > > Till
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma wrote:
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > Thanks Yun for the detailed explanation.
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > A simple supplementary explanation about the sink case:
> Maybe we
> > > > >
> > > > > could
> > > > >
> > > > > >
> > > > >
> > > > > > > use
> > > > >
> > > > > >
> > > > >
> > > > > > > > `OperatorCoordinator` to avoid sending the element to the
> > > > downstream
> > > > >
> > > > > >
> > > > >
> > > > > > > > operator.
> > > > >
> > > > > >
> > > > >
> > > > > > > > But I agree we could not limit the users not to emit records
> in
> > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > `notiyCheckpointComplete`.
> > > > >
> > > > > >
> > > > >
> > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > Best,
> > > > >
> > > > > >
> > > > >
> > > > > > > > Guowei
> > > > >
> > > > > >
> > > > >
> > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao
> > > > >
> > > > > >
> > > > >
> > > > > > > > wrote:
> > > > >
> > > > > >
> > > > >
> > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Hi all,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > I'd like to first detail the issue with emitting records
> in
> > > > >
> > > > > >
> > > > >
> > > > > > > > > notifyCheckpointComplete for context. For specific usage,
> > > > >
> > > > > >
> > > > >
> > > > > > > > > an example would be for sink, it might want to write some
> > > > metadata
> > > > >
> > > > > >
> > > > >
> > > > > > > after
> > > > >
> > > > > >
> > > > >
> > > > > > > > > all the transactions are committed
> > > > >
> > > > > >
> > > > >
> > > > > > > > > (like write a marker file _SUCCESS to the output
> directory).
> > > This
> > > > >
> > > > > > case
> > > > >
> > > > > >
> > > > >
> > > > > > > is
> > > > >
> > > > > >
> > > > >
> > > > > > > > > currently supported via the two level
> > > > >
> > > > > >
> > > > >
> > > > > > > > > committers of the new sink API: when received
> endOfInput(),
> > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > Committer
> > > > >
> > > > > >
> > > > >
> > > > > > > > > wait for another checkpoint to
> > > > >
> > > > > >
> > > > >
> > > > > > > > > commits all the pending transactions and emit the list of
> > > files
> > > > to
> > > > >
> > > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > > GlobalCommitter. The GlobalCommitter
> > > > >
> > > > > >
> > > > >
> > > > > > > > > would wait for another checkpoint to also write the
> metadata
> > > with
> > > > >
> > > > > 2pc
> > > > >
> > > > > >
> > > > >
> > > > > > > > > (Although sometimes 2pc is not needed
> > > > >
> > > > > >
> > > > >
> > > > > > > > > for writing metadata, it should be only an optimization
> and
> > > still
> > > > >
> > > > > >
> > > > >
> > > > > > > > requires
> > > > >
> > > > > >
> > > > >
> > > > > > > > > the Committer do commit before
> > > > >
> > > > > >
> > > > >
> > > > > > > > > notifying the global Committer. Also another note is
> > > > >
> > > > > GlobalCommitter
> > > > >
> > > > > > is
> > > > >
> > > > > >
> > > > >
> > > > > > > > > also added for some other cases
> > > > >
> > > > > >
> > > > >
> > > > > > > > > like some sinks want an commiter with dop = 1, like
> > > IceBergSink).
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > However, a more general issue to me is that currently we
> do
> > > not
> > > > >
> > > > > limit
> > > > >
> > > > > >
> > > > >
> > > > > > > > > users to not emit records in
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > notifyCheckpointComplete in the API level. The sink case
> could
> > > be
> > > > >
> > > > > >
> > > > >
> > > > > > > viewed
> > > > >
> > > > > >
> > > > >
> > > > > > > > > as a special case, but in addition
> > > > >
> > > > > >
> > > > >
> > > > > > > > > to this one, logically users could also implement their
> own
> > > cases
> > > > >
> > > > > > that
> > > > >
> > > > > >
> > > > >
> > > > > > > > > emits records in notifyCheckpointComplete.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Best,
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Yun
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > ------------------Original Mail ------------------
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Sender:Arvid Heise
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Send Date:Fri Feb 12 20:46:04 2021
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Recipients:dev
> > > > >
> > > > > >
> > > > >
> > > > > > > > > CC:Yun Gao
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After
> > > Tasks
> > > > >
> > > > > >
> > > > >
> > > > > > > Finished
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Hi Piotr,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Thank you for raising your concern. Unfortunately, I do
> not
> > > have
> > > > a
> > > > >
> > > > > >
> > > > >
> > > > > > > better
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > idea than doing closing of operators intermittently with
> > > > >
> > > > > checkpoints
> > > > >
> > > > > > (=
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > multiple last checkpoints).
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > However, two ideas on how to improve the overall user
> > > experience:
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > 1. If an operator is not relying on
> notifyCheckpointComplete,
> > > we
> > > > >
> > > > > can
> > > > >
> > > > > >
> > > > >
> > > > > > > > close
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > it faster (without waiting for a checkpoint). In general,
> I'd
> > > > >
> > > > > assume
> > > > >
> > > > > >
> > > > >
> > > > > > > that
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > almost all non-sinks behave that way.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > 2. We may increase the checkpointing frequency for the
> last
> > > > >
> > > > > >
> > > > >
> > > > > > > checkpoints.
> > > > >
> > > > > >
> > > > >
> > > > > > > > We
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > need to avoid overloading checkpoint storages and task
> > > managers,
> > > > >
> > > > > but
> > > > >
> > > > > > I
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > assume the more operators are closed, the lower the
> > > checkpointing
> > > > >
> > > > > >
> > > > >
> > > > > > > > interval
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > can be.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > For 1, I'd propose to add (name TBD):
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > default boolean StreamOperator#requiresFinalCheckpoint() {
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > return true;
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > }
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > This means all operators are conservatively (=slowly)
> closed.
> > > For
> > > > >
> > > > > > most
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > operators, we can then define their behavior by overriding
> in
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > AbstractUdfStreamOperator
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > @Override
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > boolean
> AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > return userFunction instanceof CheckpointListener;
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > }
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > This idea can be further refined in also adding
> > > > >
> > > > > > requiresFinalCheckpoint
> > > > >
> > > > > >
> > > > >
> > > > > > > > to
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > CheckpointListener to exclude all operators with UDFs that
> > > > >
> > > > > implement
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > CheckpointListener but do not need it for 2pc.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > @Override
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > boolean
> AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > return userFunction instanceof CheckpointListener &&
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > ((CheckpointListener)
> userFunction).requiresFinalCheckpoint();
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > }
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > That approach would also work for statebackends/snapshot
> > > > strategies
> > > > >
> > > > > >
> > > > >
> > > > > > > that
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > require some 2pc.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > If we can contain it to the @PublicEvolving
> StreamOperator, it
> > > > >
> > > > > would
> > > > >
> > > > > > be
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > better of course.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Best,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Arvid
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > wrote:
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > Hey,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > I would like to raise a concern about implementation of
> the
> > > > final
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > checkpoints taking into account operators/functions that
> are
> > > > >
> > > > > >
> > > > >
> > > > > > > > implementing
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > two phase commit (2pc) protocol for exactly-once
> processing
> > > > with
> > > > >
> > > > > > some
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > external state (kept outside of the Flink). Primarily
> > > > >
> > > > > exactly-once
> > > > >
> > > > > >
> > > > >
> > > > > > > > sinks.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > First of all, as I understand it, this is not planned in
> the
> > > > >
> > > > > first
> > > > >
> > > > > >
> > > > >
> > > > > > > > > version
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > of this FLIP. I'm fine with that, however I would
> strongly
> > > > >
> > > > > > emphasize
> > > > >
> > > > > >
> > > > >
> > > > > > > > this
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > in every place we will be mentioning FLIP-147 efforts.
> This
> > > is
> > > > >
> > > > > >
> > > > >
> > > > > > > because
> > > > >
> > > > > >
> > > > >
> > > > > > > > > me,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > as a user, upon hearing "Flink supports checkpointing
> with
> > > > >
> > > > > bounded
> > > > >
> > > > > >
> > > > >
> > > > > > > > > inputs"
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > I would expect 2pc to work properly and to commit the
> > > external
> > > > >
> > > > > side
> > > > >
> > > > > >
> > > > >
> > > > > > > > > effects
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > upon finishing. As it is now, I (as a user) would be
> > > surprised
> > > > >
> > > > > > with a
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > silent data loss (of not committed trailing data). This
> is
> > > > just a
> > > > >
> > > > > >
> > > > >
> > > > > > > > remark,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > that we need to attach this warning to every blog
> > > > >
> > > > > >
> > > > >
> > > > > > > > post/documentation/user
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > mailing list response related to "Support Checkpoints
> After
> > > > Tasks
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > Finished". Also I would suggest to prioritize the follow
> up
> > > of
> > > > >
> > > > > >
> > > > >
> > > > > > > > supporting
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 2pc.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > Secondly, I think we are missing how difficult and
> > > problematic
> > > > >
> > > > > will
> > > > >
> > > > > >
> > > > >
> > > > > > > be
> > > > >
> > > > > >
> > > > >
> > > > > > > > > 2pc
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > support with the final checkpoint.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > For starters, keep in mind that currently 2pc can be
> > > > implemented
> > > > >
> > > > > by
> > > > >
> > > > > >
> > > > >
> > > > > > > > users
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > using both `@Public` APIs as functions and
> `@PublicEvolving`
> > > > >
> > > > > >
> > > > >
> > > > > > > operators
> > > > >
> > > > > >
> > > > >
> > > > > > > > in
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > any place in the job graph. It's not limited to only the
> > > sinks.
> > > > >
> > > > > For
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > example users could easily implement the `AsynFunction`
> (for
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > `AsyncWaitOperator`) that is using 2pc based on the
> > > > >
> > > > > >
> > > > >
> > > > > > > > `CheckpointListener`
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > interface. I'm not saying it's common, probably just a
> tiny
> > > > >
> > > > > > minority
> > > > >
> > > > > >
> > > > >
> > > > > > > of
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > users are doing that (if any at all), but nevertheless
> > > that's
> > > > >
> > > > > >
> > > > >
> > > > > > > possible
> > > > >
> > > > > >
> > > > >
> > > > > > > > > and
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > currently (implicitly?) supported in Flink.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > Next complication is the support of bounded streams
> > > > >
> > > > > >
> > > > >
> > > > > > > (`BoundedOneInput`
> > > > >
> > > > > >
> > > > >
> > > > > > > > or
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > `BoundedMultiInput` interfaces) and the closing/shutdown
> > > > >
> > > > > procedure
> > > > >
> > > > > > of
> > > > >
> > > > > >
> > > > >
> > > > > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > operators. Currently it works as follows:
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 0. Task receives EndOfPartitionEvent (or source
> finishes)
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > 1. `endOfInput` is called on the first operator in the
> chain
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 2. We quiesce the processing timers
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > >
> (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> > > > for
> > > > >
> > > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > > first
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > operator, so no new timers will be triggered
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 3. We wait for the already fired timers to finish
> executing
> > > > >
> > > > > > (spinning
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > mailbox loop)
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 4. We are closing the first operator
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 5. We go to the next (second) operator in the chain and
> > > repeat
> > > > >
> > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > steps
> > > > >
> > > > > >
> > > > >
> > > > > > > > > 1.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > to 5.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > This is because operators can emit data after processing
> > > > >
> > > > > >
> > > > >
> > > > > > > `endOfInput`,
> > > > >
> > > > > >
> > > > >
> > > > > > > > > from
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > timers, async mailbox actions and inside the `close`
> method
> > > > >
> > > > > itself.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > Now the problem is to support the final checkpoint with
> 2pc,
> > > we
> > > > >
> > > > > > need
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > trigger `snapshotState` and `notifyCheckpointComplete`
> call
> > > at
> > > > >
> > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > very
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > least only after `endOfInput` call on the operator.
> Probably
> > > > the
> > > > >
> > > > > > best
> > > > >
> > > > > >
> > > > >
> > > > > > > > > place
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > would be in between steps 3. and 4. However that means,
> we
> > > > would
> > > > >
> > > > > be
> > > > >
> > > > > >
> > > > >
> > > > > > > > > forced
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > to wait for steps 1. to 3. to finish, then wait for a
> next
> > > > >
> > > > > > checkpoint
> > > > >
> > > > > >
> > > > >
> > > > > > > > to
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > trigger AND complete, before finally closing the head
> > > operator,
> > > > >
> > > > > and
> > > > >
> > > > > >
> > > > >
> > > > > > > > only
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > then we can start closing the next operator in the
> chain:
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 0. Task receives EndOfPartitionEvent (or source
> finishes)
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > 1. `endOfInput` is called on the first operator in the
> chain
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 2. We quiesce the processing timers
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > >
> (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> > > > for
> > > > >
> > > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > > first
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > operator, so no new timers will be triggered
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 3. We wait for the already fired timers to finish
> executing
> > > > >
> > > > > > (spinning
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > mailbox loop)
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > *3b. We wait for one more checkpoint to trigger and for
> the
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > `notifyCheckpointComplete` RPC.*
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 4. We are closing the first operator
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 5. We go to the next (second) operator in the chain and
> > > repeat
> > > > >
> > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > steps
> > > > >
> > > > > >
> > > > >
> > > > > > > > > 1.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > to 5.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > That means, we can close one operator per successful
> > > > checkpoint.
> > > > >
> > > > > To
> > > > >
> > > > > >
> > > > >
> > > > > > > > close
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 10 operators, we would need 10 successful checkpoints.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > I was thinking about different approaches to this
> problem,
> > > and
> > > > I
> > > > >
> > > > > >
> > > > >
> > > > > > > > couldn't
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > find any viable ones. All I could think of would break
> the
> > > > >
> > > > > current
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > `@Public` API and/or would be ugly/confusing for the
> users.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > For example a relatively simple solution, to introduce a
> > > > >
> > > > > `preClose`
> > > > >
> > > > > >
> > > > >
> > > > > > > or
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > `flush` method to the operators, with a contract that
> after
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > `flush`, operators would be forbidden from emitting more
> > > > records,
> > > > >
> > > > > > so
> > > > >
> > > > > >
> > > > >
> > > > > > > > that
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > we can replace step 4. with this `flush` call, and then
> > > having
> > > > a
> > > > >
> > > > > >
> > > > >
> > > > > > > single
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > checkpoint to finish 2pc for all of the operators inside
> the
> > > > >
> > > > > chain,
> > > > >
> > > > > >
> > > > >
> > > > > > > > > doesn't
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > work. Sheer fact of adding this `flush` method and
> changing
> > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > contract
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > would break the current API and Yun Gao has pointed out
> to
> > > me,
> > > > >
> > > > > that
> > > > >
> > > > > >
> > > > >
> > > > > > > we
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > either already support, or want to support operators
> that
> > > are
> > > > >
> > > > > >
> > > > >
> > > > > > > emitting
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > records from within the `notifyCheckpointComplete` call:
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > Yun Gao:
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > like with the new sink api there might be writer ->
> > > committer
> > > > >
> > > > > ->
> > > > >
> > > > > >
> > > > >
> > > > > > > > global
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > committer, the committer would need to wait for the last
> > > > >
> > > > > checkpoint
> > > > >
> > > > > >
> > > > >
> > > > > > > to
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > commit
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > the last piece of data, and after that it also need to
> > > emit
> > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > list
> > > > >
> > > > > >
> > > > >
> > > > > > > > of
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > transactions get committed to global committer to do
> some
> > > > >
> > > > > >
> > > > >
> > > > > > > finalization
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > logic.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > So it wouldn't solve the problem (at least not fully).
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > I don't know if anyone has any better ideas how to solve
> > > this
> > > > >
> > > > > >
> > > > >
> > > > > > > problem?
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > Piotrek
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > pt., 15 sty 2021 o 14:57 Yun Gao
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > napisał(a):
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > Hi Aljoscha,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > I think so since we seems to do not have other
> divergence
> > > and
> > > > >
> > > > > new
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > objections now. I'll open the vote then. Very thanks!
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > Best,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > Yun
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> ------------------------------------------------------------------
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > From:Aljoscha Krettek
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > Send Time:2021 Jan. 15 (Fri.) 21:24
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > To:dev
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints
> After
> > > > Tasks
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Finished
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > Thanks for the summary! I think we can now move
> towards a
> > > > >
> > > > > [VOTE]
> > > > >
> > > > > >
> > > > >
> > > > > > > > > thread,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > right?
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > On 2021/01/15 13:43, Yun Gao wrote:
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > >1) For the problem that the "new" root task
> coincidently
> > > > >
> > > > > > finished
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > > >before getting triggered successfully, we have listed
> two
> > > > >
> > > > > > options
> > > > >
> > > > > >
> > > > >
> > > > > > > in
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > >the FLIP-147[1], for the first version, now we are
> not
> > > tend
> > > > to
> > > > >
> > > > > > go
> > > > >
> > > > > >
> > > > >
> > > > > > > > with
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > >the first option that JM would re-compute and
> re-trigger
> > > new
> > > > >
> > > > > >
> > > > >
> > > > > > > sources
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > >when it realized some tasks are not triggered
> > > successfully.
> > > > >
> > > > > This
> > > > >
> > > > > >
> > > > >
> > > > > > > > > option
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > >would avoid the complexity of adding new PRC and
> > > duplicating
> > > > >
> > > > > > task
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > >states, and in average case it would not cause too
> much
> > > > >
> > > > > > overhead.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > > You wrote "we are *not* tend to go with the first
> option",
> > > > but
> > > > >
> > > > > I
> > > > >
> > > > > >
> > > > >
> > > > > > > > think
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > you meant wo write "we tend to *now* go with the first
> > > > option",
> > > > >
> > > > > >
> > > > >
> > > > > > > > right?
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > That's also how it is in the FLIP, I just wanted to
> > > clarify
> > > > for
> > > > >
> > > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > mailing list.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > >
> > > >
> > >
> > >
> >
> >
> >
>
>

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

Posted by Piotr Nowojski <pn...@apache.org>.
Hi Kezhu,

What do you mean by “end-flushing”? I was suggesting to just keep
`endOfInput()` and `dispose()`. Are you suggesting to have a one
`endFlushing()` method, that is called after quiescing timers/mailbox, but
before final checkpoint and `dispose()`? Are we sure we really need this
extra call? Note. If we don't need it at the moment, we could always
introduce it in the future, while if we don't and won't need it, why
complicate the API?

About the idea of returning the global-commit-handle from
`notifyCheckpointComplete()` call. Note it will be more difficult to
implement, as `CheckpointCoordinator` will need to have one extra stage of
waiting for some actions to complete. Implementation will probably be
easier if we return the global-commit-handle from `snapshotState()` call.

Also, `global-commit-handles` will need to be part of the checkpoint. They
will need to be restored/recovered in case of failure. Because of that it
might be actually impossible to implement those handles as returned from
`notifyCheckpointComplete()`. In this solution we would be in a precarious
position if the main checkpoint succeeded, CheckpointCoordinator would
start issuing `notifyCheckpointComplete()`, but persisting of the handles
would fail/keep failing. How would we recover from such a situation? We can
not recover to a previous checkpoint (`notifyCheckpointComplete()` were
already issued), but at the same time the current checkpoint is not fully
completed (global-commit-handles can not be checkpointed).

Best,
Piotrek



czw., 4 mar 2021 o 06:33 Kezhu Wang <ke...@gmail.com> napisał(a):

> Hi all,
>
> Glad to see convergence here and FLINK-21133:
> 1. We all prefer single final checkpoint for task not individual
> checkpoints for each operators.
> 2. To above goal, if we have to breaking something, we will.
> 3. Don’t allow recording emitting in `notifyCheckpointComplete`.
>
> For “end-flushing”, I think both approaches should function in reality,
> but we also have options/responsibilities to mitigate effect of breaking
> changes:
> A. Allowing checkpoint after “close”. Introduce config option to forbid
> this during migrating releases.
> B. Renaming “close” to “other-end-flushing-method”. We can backport that
> newly introducing “end-flushing”(as empty default method) to earlier
> releases in following patch releases. The backporting “end-flushing” will
> be called just before “close” in future patch releases. We could call
> “close” just before “dispose" in future releases and `final` it in
> `AbstractStreamOperator` when ready(to break user side code).
>
> If breaking change for this “end-flushing” in inevitable, I am kind of
> prefer renaming and backport approach. It is a chance for us to rethink the
> whole thing and discard misleading “close”(currently it is mixed/misused
> with “end-flushing” and “cleanup-resource” though javadoc claims only
> “end-flushing, this could also be considered as a bug though).
>
> Besides this, will FLIP-147 eventually need some ways to decide whether an
> operator need final checkpoint @Yun @Guowei ?  @Arvid mentions this in
> earlier mail.
>
>
> For the two phase commit, @Piotrek I like your idea. I think that
> “commit-handle” could be return to checkpoint-coordinator through
> `notifyCheckpointComplete`. This way that “commit-handle” might be reused
> by operator-coordinator’s `notifyCheckpointComplete`. Suppose following
> changes:
>
> 1. `CompletableFuture<SerializedValue<?>> notifyCheckpointCompleteAsync()`
> in operator.
> 2. `CompletableFuture<Void> notifyCheckpointCompleteAsync(Map<Integer,
> CompletableFuture<SerializedValue>> subtasks)` in operator coordinator.
>
> These changes need support from:
> * Checkpoint coordinator level to bridge operator and coordinator through
> task
> * Operator level to compat existing `notifyCheckpointComplete`
>
> The checkpoint procedure will looks like:
> 1. Trigger checkpoint for operator coordinator.
> 2. If above succeeds, trigger tasks checkpoint. Abort otherwise.
> 3. If all above succeeds, complete current checkpoint. Abort otherwise.
> 4. If job fails after, restore from above “completed” checkpoint.
> 5. Notify checkpoint completion to tasks.
> 6. Notify checkpoint completion to coordinators.
> 7. Wait step#5 and step#6 to succeed. Now it is real completed. Either
> this succeed or job failed in meantime ? May be other concurrent conditions.
>
> With these changes, migration FLIP-143 sink to operator coordinator should
> be easy.
>
> It will definitely complicate currently complex checkpoint coordinator as
> @Till mentioned in FLINK-21133.
>
>
> Best,
> Kezhu Wang
>
> On March 3, 2021 at 01:09:50, Piotr Nowojski (pnowojski@apache.org) wrote:
>
> Hi,
>
> Thanks for reminding me. I think FLIP-147 will have to deal in one way or
> another with the (re?)emitting MAX_WATERMARK. There is a pre-existing
> issue
> that watermarks are not checkpointed/stored on state, and there was/is now
> clear answer how we should handle this as far as I remember. One
> problematic case are two/multiple input tasks or UnionInputGate, where
> combined watermark is the min of all inputs (held in memory). The problem
> so far is a bit benign, as after recovery we are losing the combined
> watermark value, but it's being slowly/lazily restored, as new watermarks
> are sent from the sources. With finished sources that won't be a case.
>
> I've spent more time thinking about the two phase commit issue, and I
> agree
> it would be best to have a single final checkpoint for the operators. The
> more I think about it, the more I'm leaning toward the
> "OperatorCoordinator" approach. Yes, it would need the extra complexity of
> running user code on the JobManager, but that's already at least partially
> done (FLIP-27). On the other hand it would allow us to keep the runtime
> code cleaner and simpler I think. It's still an invasive change.
>
> 1. We forbid emitting records from
> `notifyCheckpointComplete`/`snapshotState` calls, or at the very least
> from
> the final calls.
> 2. We need to clarify/clean up contracts for both operators AND user
> functions when it comes to `endOfInput()`, `close()` and `dispose()`
> methods. For example:
> a) we keep `endOfInput()` on the operators level and add an equivalent of
> it to the user functions
> b) we move the "flushing buffered records" part of the `close()` contract
> to `endOfInput()`
> c) we remove `close()` methods altogether (IMO it's better to have an
> explicit false conflict, rather than a silent real one)
> d) we keep `dispose()` on the operator level for release resources, and
> we add such method to the user functions
> 3. We change (or add another version) a return type of either `void
> snapshotState(...)` or `void notifyCheckpointComplete(...)` calls, so that
> they will return some kind of `GlobalCommitHandle`.
>
> Especially this point 2. is intrusive. Now, putting all of that together.
> When a task receives a logical "end of input" signal (current
> `EndOfPartitionEvent`), it begins shutting down procedure (let's assume we
> create `GlobalCommitHandle` in `snapshotState`).
>
> 0. n = 1
> 1. endOfInput on the n'th operator
> 2. quisec timers and mailbox executor for n'th operator
> // after this point n'th operator is forbidden to emit any new records
> 3. n += 1, and go to 1., until we handle all of the operators
> 4. wait for final checkpoint, issue `snapshotState` and collect
> `GlobalCommitHandle`s and send them to the JM
> 5. JM/CheckpointCoordinator collects all `GlobalCommitHandles`. They are
> an
> integral part of the checkpoint. They could be added to the respective
> `OperatorCoordinator`s via `handleEventFromOperator` as an
> `OperatorEvent`.
> 6. When checkpoint is considered completed, `notifyCheckpointComplete` are
> being issued, both on the operators, and `OperatorCoordinator` - local
> commit and global commits would be happening at the same time, or even
> global commit code would be executed before local final commit in this
> version.
> 7. Only now can we finally dispose of all operators in the task.
>
> If there is a failure, especially before we manage to perform all
> `notifyCheckpointComplete` calls (for example on the
> `OperatorCoordinator`), we would need to recover from last "completed"
> checkpoint, recover attached `GlobalCommitHandle`s , and re-commit them.
> It
> also means we need to recover already finished operators, that will never
> process any records, just to issue the final `notifyCheckpointComplete`
> call and make sure they commit their external side effects.
>
> There are at least a couple of questions about this general idea:
> I. Naming of the methods (I suggested to drop `close()` and keep in the
> operators and introduce in the functions `endOfInput()` and `dispose()`
> II. When should the `GlobalCommitHandle` be created? Should it be returned
> from `snapshotState()`, `notifyCheckpointComplete()` or somewhere else?
> III. What should be the ordering guarantee between global commit and local
> commit, if any? Actually the easiest to implement would be undefined, but
> de facto global commit happening before local commits (first invoke
> `notifyCheckpointComplete()` on the `OperatorCoordinator` and either after
> or in parallel send `notifyCheckpointComplete()` RPCs). As far as I can
> tell, undefined order should work for the use cases that I'm aware of.
> IV. Can we reuse `OperatorCoordinator` abstraction for this purpose?
> Currently it's being used to induce checkpoints with FLIP-27 sources as
> far
> as I understand, here we would be using it for different purposes.
>
> Best,
> Piotrek
>
> pon., 1 mar 2021 o 07:30 Yun Gao <yu...@aliyun.com> napisał(a):
>
> > (Sorry that I repeat this mail since the last one is not added into the
> > same mail list thread,
> > very sorry for the inconvenience)
> >
> > Hi all,
> >
> > Very thanks for all the deep thoughts!
> >
> > > How to implement the stop-with-savepoint --drain/terminate command
> with
> > > this model: One idea could be to tell the sources that they should
> stop
> > > reading. This should trigger the EndOfPartitionEvent to be sent
> > > downstream.
> > > This will transition all operators into the TERMINATING state.
> > >
> > > Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
> > > returned. To achieve above, possible works should be required:
> > > * Promote `EndOfPartitionEvent` from `Task` to `StreamTask`. This may
> > > have some interferences with BatchTask or network io stack.
> > > * Or introducing stream task level `EndOfUserRecordsEvent`(from
> PR#14831
> > > @Yun @Piotr)
> > > * Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.
> >
> > I also have similar concern with Kezhu for the issue whether do we need
> to
> > introduce a new message
> > to notify the operators to endOfInput/close ? The main concerns of
> reusing
> > that EndOfPartitionEvent
> > is that
> > 1. The EndOfPartitionEvent is currently emitted in Task instead of
> > StreamTask, we would need some
> > refactors here.
> > 2. Currently the InputGate/InputChannel would be released after the
> > downstream tasks have received
> > EndOfPartitionEvent from all the input channels, this would makes the
> > following checkpoint unable to
> > perform since we could not emit barriers to downstream tasks ?
> >
> > Regarding the MAX_WATERMARK, I still not fully understand the issue
> since
> > it seems to me
> > that now Flink won't snapshot the watermark now? If the job failover,
> the
> > window operator
> > would reload all the pending windows before flushed by MAX_WATERMARK and
> > when the
> > job finish again, it would re-emit the MAX_WATERMARK?
> >
> > Best,
> > Yun
> >
> >
> > ------------------------------------------------------------------
> > From:Kezhu Wang <ke...@gmail.com>
> > Send Time:2021 Mar. 1 (Mon.) 01:26
> > To:Till Rohrmann <tr...@apache.org>
> > Cc:Piotr Nowojski <pi...@gmail.com>; Guowei Ma <
> > guowei.mgw@gmail.com>; dev <de...@flink.apache.org>; Yun Gao <
> > yungao.gy@aliyun.com>; jingsonglee0@gmail.com <ji...@gmail.com>
> > Subject:Re: Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > Finished
> >
> > In “stop-with-savepoint —drain”, MAX_WATERMARK is not an issue. For
> normal
> > finishing task, not allowing unaligned checkpoint does not solve the
> > problem as MAX_WATERMARK could be persisted in downstream task. When
> > scenario @Piotr depicted occurs, downstream(or further downstream)
> window
> > operator will count all inputs as late.
> >
> > > If we ensure that the MAX_WATERMARK is only persisted in state if a
> > recovery will trigger immediately the shut down of this operator, then
> it
> > shouldn't be an issue.
> >
> > You are right in case the assumption holds, I have same thought as you
> > before. But I am kind of worry about whether it is too prefect to be
> > fragile. This requires strong guarantee from implementation that a
> recovery
> > from TERMINATING stage should go directly to that stage.
> >
> > > I think the MAX_WATERMARK event should be sent either just before or
> with
> > the EndOfPartitionEvent.
> >
> > I prefer “with the EndOfPartitionEvent”. EndOfPartitionEvent itself
> already
> > carry what ending MAX_WATERMARK try to express. May be we can reuse it ?
> > @Piotr
> >
> > A preview work[1] from @Yun in PR#14831 explains EndOfPartitionEvent as
> > checkpoint barrier if there are pending checkpoints.
> >
> >
> > [1]:
> >
> >
> https://github.com/gaoyunhaii/flink/commit/1aaa80fb0afad13a314b06783c61c01b9414f91b#diff-d4568b34060bc589cd1354bd125c35aca993dd0c9fe9e4460dfed73501116459R177
> >
> >
> > Best,
> > Kezhu Wang
> >
> > On February 28, 2021 at 21:23:31, Till Rohrmann (trohrmann@apache.org)
> > wrote:
> >
> > I think you are right with the problem of endOfInput. endOfInput should
> not
> > be used to commit final results. In fact if this termination fails then
> we
> > might end up in a different outcome of the job which is equally valid as
> > the one before the failure.
> >
> > Concerning unaligned checkpoints, I think they don't play well together
> > with draining a streaming pipeline. The problem is that in the draining
> > case you want to process all records which are still in flight but
> > unaligned checkpoints don't guarantee this as they can jump in flight
> > records.
> >
> > I think the MAX_WATERMARK event should be sent either just before or
> with
> > the EndOfPartitionEvent. If we ensure that the MAX_WATERMARK is only
> > persisted in state if a recovery will trigger immediately the shut down
> of
> > this operator, then it shouldn't be an issue.
> >
> > Cheers,
> > Till
> >
> > On Sun, Feb 28, 2021 at 9:06 AM Kezhu Wang <ke...@gmail.com> wrote:
> >
> > > Hi Till,
> > >
> > > Just for bookkeeping, some observations from current implementation.
> > >
> > > > With this model, the final checkpoint is quite simple because it is
> > > ingrained in the lifecycle of an operator. Differently said an
> operator
> > > will only terminate after it has committed its side effects and seen
> the
> > > notifyCheckpointComplete message (if it is stateful).
> > >
> >
> > > Currently, we could not mark this operator(or subtask) as terminated
> since
> > > result of `notifyCheckpointComplete`(possible side effect committing)
> is
> > > not taken into account of the belonging checkpoint. The job has to run
> to
> > > next safe point(finished or next checkpoint success) to be marked as
> > > “terminated”.
> > >
> > > > How to implement the stop-with-savepoint --drain/terminate command
> with
> > > this model: One idea could be to tell the sources that they should
> stop
> > > reading. This should trigger the EndOfPartitionEvent to be sent
> > > downstream.
> > > This will transition all operators into the TERMINATING state.
> > >
> > > Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
> > > returned. To achieve above, possible works should be required:
> > > * Promote `EndOfPartitionEvent` from `Task` to `StreamTask`. This may
> > > have some interferences with BatchTask or network io stack.
> > > * Or introducing stream task level `EndOfUserRecordsEvent`(from
> PR#14831
> > > @Yun @Piotr)
> > > * Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.
> > >
> > > Besides this, I would like to quote some discussion from FLINK-21467
> > > between @Piotr and me:
> > >
> > > From @Piotr
> > > > Note, that it's not only that endOfInput can be called multiple
> times.
> >
> > > There is a very remote possibility that the following scenario will
> happen:
> > > 1. checkpoint is taken (successfully)
> > > 2. sources are finishing
> > > 3. endOfInput is issued
> > > 4. job fails
> > > 5. job restarts to checkpoint 1.
> > > 6. after failover, because of some non deterministic logic in the
> source,
> > > sources are not finishing
> > >
> > > From me
> > > > But I think there is little work Flink can do to cope with this kind
> of
> > > issues. The checkpoint could be a savepoint triggered from user side
> and
> > > the "non deterministic logic" could be a change from user(eg. changing
> of
> > > stoppingOffsets in KafkaSource).
> > >
> >
> > > > I think the "non deterministic logic" could cause trouble in
> combination
> > > with unaligned checkpoint and downstream window operator. Unaligned
> > > checkpoint will persist "MAX_WATERMARK" in state, after restarting,
> > > "MAX_WATERMARK" will cause downstream window operator ignores all
> future
> > > inputs.
> > >
> > > FLIP-147 demands no new records from end-of-stream-flushing, but
> source
> > > will emit “MAX_WATERMARK” on ending. Previously, I thought it is not a
> >
> > > valid issue, but turn out that it could cause trouble under scenario
> listed
> > > by @Piotr if I am not wrong.
> > >
> > >
> > > PR#14831: https://github.com/apache/flink/pull/14831
> > > FLINK-21467: https://issues.apache.org/jira/browse/FLINK-21467
> > >
> > >
> > > Best,
> > > Kezhu Wang
> > >
> > > On February 27, 2021 at 18:12:20, Till Rohrmann (trohrmann@apache.org)
>
> > > wrote:
> > >
> > > Thanks for all your thoughts. I think we should further think through
> >
> > > whether to allow checkpoints after an operator has emitted all its
> records
> > > (e.g. after close is called currently) or not. I think by doing this
> we
> > > would nicely decouple the checkpoint taking from the operator
> lifecycle
> > > and
> > > wouldn't need special checkpoints/savepoints for the final checkpoint
> and
> > > stop-with-savepoint --drain. Let me try to explain this a bit more
> > > detailed.
> > >
> > > If we say an operator has the rough lifecycle RUNNING => TERMINATING
> =>
> > > TERMINATED where we go from RUNNING into TERMINATING after we have
> seen
> > > the
> > > EndOfPartitionEvent and flushed all our records. The operator goes
> from
> > > TERMINATING => TERMINATED if it has persisted all its possible side
> > > effects. Throughout all states, it is possible to trigger a
> checkpoint. A
> > > stateless operator will immediately go from TERMINATING to TERMINATED
> > > whereas a stateful operator would wait for another checkpoint to be
> > > triggered and successfully completed (notifyCheckpointComplete).
> > >
> > > With this model, the final checkpoint is quite simple because it is
> > > ingrained in the lifecycle of an operator. Differently said an
> operator
> > > will only terminate after it has committed its side effects and seen
> the
> >
> > > notifyCheckpointComplete message (if it is stateful). Here it is
> important
> > > to note that in the streaming case, different bounded operators can
> > > terminate at different times. They don't have to terminate all with
> the
> > > same checkpoint.
> > >
> > > How to implement the stop-with-savepoint --drain/terminate command
> with
> > > this model: One idea could be to tell the sources that they should
> stop
> > > reading. This should trigger the EndOfPartitionEvent to be sent
> > > downstream.
> >
> > > This will transition all operators into the TERMINATING state. Next
> the JM
> > > can trigger a checkpoint to shut the operators down and not to wait
> for
> > > the
> > > next automatic checkpoint trigger event.
> > >
> > > By allowing checkpoints throughout the entire lifecycle of an operator
> we
> > > disallow sending records from notifyCheckpointComplete because this
> > > message
> > > will also be sent in the state TERMINATING where an operator has
> already
> > > produced all of its records.
> > >
> > > What do you think? Maybe this model overlooks some important cases.
> One
> > > downside is that we will break the operator API with changing the
> > > lifecycle
> > > of an operator.
> > >
> > > Cheers,
> > > Till
> > >
> > >
> > >
> > > On Fri, Feb 26, 2021 at 7:29 AM Kezhu Wang <ke...@gmail.com> wrote:
> > >
> > > > Hi Yun,
> > > >
> > > > The termination phase I drafted depends on a de-fact that
> >
> > > > `notifyCheckpointComplete` is ignored currently after `close`. Thus,
> any
> > > > external materialization inside `close`(including `endInput`) or
> > > > `notifyCheckpointComplete` is either not guaranteed or not
> committed.
> > > >
> > > > I tried to emphasize no-breaking changes and post-pone migration in
> > > later
> > > > releases for this termination phase. But now, I am kind of worry
> about
> > > > whether it will cause long-term maintenance hell.
> > > >
> > > > Personally, I think allowing checkpoint after `close`(@Till proposed
> > > this
> > > > in FLINK-21133) could be the minimal change. But there are concerns:
> > > > * It will break some existing code possibly in a silent way.(@Piotr
> > > pointed
> > > > this already in FLINK-21133)
> > > > * I think the name `close` is kind of misleading. (@Piotr suggested
> > > > renaming this to `finish` to not break code silently in FLINK-21133)
> > > >
> > > > > stop-with-savepoint --drain would wait for a specific savepoint
> > > >
> > > > For stop-with-savepoint, the checkpoint barrier is already created
> or
> > > > received there.
> > > >
> > > >
> > > > Best,
> > > > Kezhu Wang
> > > >
> > > > On February 25, 2021 at 21:02:30, Yun Gao (yungao.gy@aliyun.com
> > ) wrote:
> > > >
> > > > Hi all,
> > > >
> > > > Very thanks for the discussions!
> > > >
> > > >
> > > >
> > > > A. Regarding how to avoid emitting records in
> notifyCheckpointComplete:
> > > >
> > > > Currently the structure of a new sink is writer -> committer ->
> global
> > > > committer and the paralellism of global committer
> > > > must be one. By design it would be used in several cases:
> > > > 1. writer -> committer: for normal sinks that write pending data in
> > > writer
> > > > and commit pending data in committer.
> > > > 2. writer -> global committer: for sinks require the committer's
> > > > parallelism be 1, like iceberg sink which
> > > > relies on optimistic lock to commit data hopes to reduce the
> conflicts.
> > > > 3. writer -> committer -> global committer: for sinks that also need
> to
> > > > write meta data (like _SUCCESS file
> > > > or add record in hive metastore) in global committer.
> > > >
> > > > The third case would cause the cascade commit problem. To overcome
> this
> > > > problem, we would like to
> > > > 1. Previously the global committer also support writing metadata
> with
> > > 2pc.
> > > > Now we disable this mode
> > > > and always rely on the property that writing metadata is repeatable.
> > > With
> > > > this limitation there should no
> > > > need of cascade commit, but the committer would still emit records
> in
> > > > notifyCheckpointComplete.
> > > > 2. We further move global committer in the case 3 to the operator
> > > > coordinator. Then the committer would
> > > > not need to emit records, but communicates with its operator
> > > coordinator.
> > > >
> > > > One core issue for using the OperatorCoordinator is how to keep the
> > > > communication between the operator
> > > > and the OperatorCoordinator exactly-once. Since the message is
> always
> > > from
> > > > the oeprator to the OperatorCoordinator
> > > > in this case, we would only need to bookkeep the message sent
> between
> > > the
> > > > OperatorCoordinator takes snapshot and
> > > > the Operator takes snapshot in the state of the Operator.
> > > >
> > > > On how to achieve the change in detail we would still need some
> think,
> > > it
> > > > currently seems we would have to had
> > > > some modification to the current new sink api.
> > > >
> > > >
> > > >
> > > > B. Regarding the stop-with-savepoint --drain
> > > >
> > > > Very thanks @Piotr for the further explanation and now I realize I
> have
> > > > understand wrongly for the semantics of
> > > > stop-with-savepoint --drain. Now I think that the problem should be
> we
> > > > should also include the records produced in
> > > > `endOfInput()` and `close()` also in the last savepoint, am I
> correct?
> > > If
> > > > so, it seems we still have some undetermined options for
> > > > the lifecycle of the operator, like in Kezhu's proposal the close()
> > > happens
> > > > at last, but it seems close() might also emit records (
> > > > so now the operator are closed with op1's close() -> op2's
> endOfInput()
> > > ->
> > > > op2's close() -> op3's endOfinput -> ...) ?
> > > >
> > > > And on the other side, as Kezhu has also proposed, perhapse we might
> > > have
> > > > the stop-with-savepoint --drain and normal exit in the same process,
> > > > but have slightly difference in that stop-with-savepoint --drain
> would
> > > wait
> > > > for a specific savepoint and in normal exit, the operator
> > > > might wait for arbitrary checkpoint. If we could achieve not
> emitting
> > > > records in notifyCheckpointComplete, stop-with-savepoint --drain
> could
> > > > be done with one savepoint, and for the normal exit, the operator
> would
> > > not
> > > > need to wait for other slow operators to exit.
> > > >
> > > > Best,
> > > > Yun
> > > >
> > > >
> > > >
> > > > ------------------Original Mail ------------------
> > > > *Sender:*Kezhu Wang <ke...@gmail.com>
> > > > *Send Date:*Thu Feb 25 15:11:53 2021
> > > > *Recipients:*Flink Dev <de...@flink.apache.org>, Piotr Nowojski <
> > > > piotr.nowojski@gmail.com>
> > > > *CC:*Guowei Ma <gu...@gmail.com>, jingsonglee0@gmail.com <
> > > > jingsonglee0@gmail.com>
> > > > *Subject:*Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After
> Tasks
> > > > Finished
> > > >
> > > > > Hi all, thanks for driving this and especially Piotr for re-active
> > > this
> > > > >
> > > > > thread.
> > > > >
> > > > >
> > > > >
> > > > > First, for `notifyCheckpointComplete`, I have strong preference
> > > towards
> > > > >
> > > > > "shut down the dataflow
> > > > >
> > > > > pipeline with one checkpoint in total", so I tend to option
> dropping
> > > > "send
> > > > >
> > > > > records" from
> > > > >
> > > > > `notifyCheckpointComplete` for next level committing in pipeline,
> if
> > > we
> > > > >
> > > > > ever support it. Without this,
> > > > >
> > > > > we are unable to stop a pipeline manually with all results
> > > materialized.
> > > > >
> > > > >
> > > > >
> > > > > Second, for shutdown unification of `stop-with-savepoint --drain`
> and
> > > > >
> > > > > FLIP-147, I draft following
> > > > >
> > > > > phase based on emerging proposals(mainly by Piotr and Till) in
> this
> > > > thread
> > > > >
> > > > > and FLINK-21133.
> > > > >
> > > > >
> > > > >
> > > > > ```java
> > > > >
> > > > > // StreamTask termination phase
> > > > >
> > > > > finish()(Call StreamOperator.finish in chaining order)
> > > > >
> >
> > > > > advanceToEndOfEventTime()(nop if for no-source inputs, this could
> also
> > > be
> > > > >
> > > > > done in input processor or finish ?)
> > > > >
> > > > > if (there-is-a-pending-terminate-savepoint) {
> > > > >
> > > > > triggerBarrierForDownStream();
> > > > >
> > > > > waitCheckpointComplete();
> > > > >
> > > > > } else if (require-two-phase-commit-in-shutdown) {
> > > > >
> > > > > waitFinalCheckpoint();
> > > > >
> > > > > waitCheckpointComplete();
> > > > >
> > > > > }
> > > > >
> > > > > // TODO: Ignore/Cancel all checkpoints from this point(or hit to
> > > > >
> > > > > coordinator in final checkpoint response ?).
> > > > >
> > > > > close();
> > > > >
> > > > > ```
> > > > >
> > > > >
> > > > >
> > > > > Branches in `if` could reside in different code paths, but the
> effect
> > > > >
> > > > > should be similar to above.
> > > > >
> > > > > The divergence reflects fact that "stop-with-savepoint --drain" is
> > > > >
> > > > > triggered already while we need
> > > > >
> > > > > to wait for final checkpoint in natural input exhausting.
> > > > >
> > > > >
> > > > >
> > > > > With carefully chosen default functions, we will not break
> existing
> > > > >
> > > > > interfaces.
> > > > >
> > > > > * `StreamOperator.finish`
> > > > >
> > > > > * `RichFunction.finish`
> > > > >
> > > > >
> > > > >
> > > > > For existing codes:
> > > > >
> > > > > 1. "stop-with-savepoint" with no migration, it should behaves as
> > > before.
> > > > >
> > > > > 2. "require-two-phase-commit-in-shutdown" evaluates to false and
> no
> > > > >
> > > > > migration, it is same as before.
> > > > >
> > > > > 3. "require-two-phase-commit-in-shutdown" evaluates to true and no
> > > > >
> > > > > migration, records in `close` will
> > > > >
> > > > > lose.
> > > > >
> > > > >
> > > > >
> > > > > For case#3:
> > > > >
> > > > > A. No external materialization in `StreamOperator.close`: Nothing
> to
> > > > lose.
> > > > >
> > > > > B. External materialization only in `StreamOperator.close`: I
> don't
> > > think
> > > > >
> > > > > Flink ever claimed that there is
> > > > >
> > > > > any guarantee for this situation.
> > > > >
> > > > > C. External materialization in `notifyCheckpointComplete` from
> > > > >
> > > > > `StreamOperator.close`: But due to fact that
> > > > >
> > > > > `notifyCheckpointComplete` was ignored after operator
> > > > >
> > > > > closed(FLINK-16383), so there will be no external
> > > > >
> > > > > materialization to lose.
> > > > >
> > > > >
> > > > >
> > > > > Then, we could recommend users to migrate possible
> > > > “end-of-stream-flushing”
> > > > >
> > > > > from “close" to “finish”.
> > > > >
> > > > >
> > > > >
> > > > > Best,
> > > > >
> > > > > Kezhu Wang
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On February 24, 2021 at 23:47:36, Piotr Nowojski (
> > > > piotr.nowojski@gmail.com
> > > > > )
> > > > >
> > > > > wrote:
> > > > >
> > > > >
> > > > >
> > > > > Thanks for the reponses Guowei and Yun,
> > > > >
> > > > >
> > > > >
> > > > > Could you elaborate more/remind me, what does it mean to replace
> > > emitting
> > > > >
> >
> > > > > results from the `notifyCheckpointComplete` with
> `OperatorCoordinator`
> > > > >
> > > > > approach?
> > > > >
> > > > >
> > > > >
> > > > > About the discussion in FLINK-21133 and how it relates to
> FLIP-147.
> > > You
> > > > are
> > > > >
> >
> > > > > right Yun gao, that in case of `stop-with-savepoint --drain` the
> whole
> > > > job
> > > > >
> > > > > finishes, while in FLIP-147 some of the sources can work for
> > > arbitrarily
> > > > >
> >
> > > > > long time after others have already finished. However from the
> runtime
> > > > >
> > > > > perspective, especially how to close a Task and it's operators,
> there
> > > is
> > > > no
> > > > >
> > > > > difference between those two cases. In both cases, we need to end
> > > input,
> > > > >
> >
> > > > > shut down processing time timers and commit external side effects
> (two
> > > > >
> > > > > phase commit support) before exiting the task. That's how the
> > > discussion
> > > > >
> > > > > about the behaviour of "stop-with-savepoint" was connected with
> > > FLIP-147.
> > > > >
> > > > >
> > > > >
> > > > > Currently on master, "stop-with-savepoint --drain" drains/flushes
> > > > buffered
> > > > >
> >
> > > > > records and deals correctly with timers, but all of that happens
> AFTER
> > > > >
> > > > > savepoint was completed. So any records flushed from the operators
> > > during
> > > > >
> > > > > endOfInput/close/shutting down processing timers are never
> committed
> > > to
> > > > >
> > > > > external systems. This is exactly the same problem as the "two
> phase
> > > > >
> >
> > > > > commit" problem of FLIP-147, that should have the same solution
> and it
> > > > >
> >
> > > > > should be solved at the same time. For example if we go with the
> *very
> > > > >
> > > > > undesirable* "one closed operator per one completed
> > > > checkpoint/savepoint",
> > > > >
> > > > > in both cases CheckpointCoordinator, Scheduler and Task would need
> to
> > > > keep
> > > > >
> > > > > the task alive and keep triggering checkpoints for that task,
> until
> > > all
> > > > >
> > > > > operators in the operator chain are closed (one closed operator
> per
> > > one
> > > > >
> > > > > completed checkpoint).
> > > > >
> > > > >
> > > > >
> > > > > Piotrek
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > śr., 24 lut 2021 o 11:30 Yun Gao napisał(a):
> > > > >
> > > > >
> > > > >
> > > > > > Hi Till, Guowei,
> > > > >
> > > > > >
> > > > >
> > > > > > Very thanks for initiating the disucssion and the deep thoughts!
> > > > >
> > > > > >
> > > > >
> >
> > > > > > For the notifyCheckpointComplete, I also agree we could try to
> avoid
> > > > >
> > > > > > emitting new records in notifyCheckpointComplete via using
> > > > >
> > > > > > OperatorCoordinator
> > > > >
> > > > > > for new sink API. Besides, the hive sink might also need some
> > > > >
> > > > > modification
> > > > >
> > > > > > for it also emits records in notifyCheckpointComplete.
> > > > >
> > > > > >
> > > > >
> >
> > > > > > For unifying the process of stopping with savepoint and finished
> due
> > > to
> > > > >
> > > > > > all records
> > > > >
> > > > > > are processed, I also agree with that unifying would always be
> > > better
> > > > if
> > > > >
> > > > > > we could achieve,
> > > > >
> > > > > > but I'm still not fully catch up with the implementation: Based
> on
> > > the
> > > > >
> > > > > > discussion in FLINK-21133,
> > > > >
> >
> > > > > > my understanding is that for stopping with savepoint, now we
> want to
> > > > >
> > > > > first
> > > > >
> > > > > > stop the source, then we
> > > > >
> > > > > > trigger a savepoint, and after the source received
> > > > >
> > > > > > notifyCheckpointComplete, the source would
> > > > >
> >
> > > > > > start emitting EndOfPartitionEvent to finish the job, am I
> correct ?
> > > > >
> > > > > >
> > > > >
> > > > > > For normal finish, a difference to me might be if we have
> multiple
> > > > >
> > > > > > sources, we could not guarantee
> > > > >
> > > > > > when the sources are to finish. We might have one source run one
> 1
> > > > minute
> > > > >
> > > > > > and another one run for
> > > > >
> >
> > > > > > 1 hour. To unify with the process with stop with savepoint, we
> might
> > > > need
> > > > >
> > > > > > to hold the fast source until
> > > > >
> > > > > > all the sources are finished? An coordinator would be introduced
> to
> > > > count
> > > > >
> > > > > > the number of sources
> > > > >
> >
> > > > > > runing and trigger the final savepoint / checkpoint. For the
> extreme
> > > > >
> > > > > > cases, if we have both bounded and
> > > > >
> > > > > > unbounded sources, we might only count how much bounded source
> are
> > > > >
> > > > > > remaining ? And if all the bounded
> > > > >
> > > > > > sources are finished we would trigger the special checkpoint.
> After
> > > all
> > > > >
> > > > > > the bounded part of the graph are
> > > > >
> > > > > > finished, the the remaining part could still do checkpoint and
> > > commit
> > > > >
> > > > > data
> > > > >
> > > > > > with FLIP-147.
> > > > >
> > > > > >
> > > > >
> > > > > > Best,
> > > > >
> > > > > > Yun
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > ------------------Original Mail ------------------
> > > > >
> > > > > > Sender:Guowei Ma
> > > > >
> > > > > > Send Date:Wed Feb 24 17:35:36 2021
> > > > >
> > > > > > Recipients:dev
> > > > >
> > > > > > CC:Arvid Heise
> > > > >
> > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > > > Finished
> > > > >
> > > > > > Hi, Till
> > > > >
> > > > > >
> > > > >
> > > > > > Thank you very much for your careful consideration
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > *1. Emit records in `NotifyCheckpointComplete`.*
> > > > >
> > > > > >
> > > > >
> > > > > > Sorry for making you misunderstanding because of my expression.
> I
> > > just
> > > > >
> > > > > >
> > > > >
> > > > > > want to say the current interface does not prevent users from
> doing
> > > it.
> > > > >
> > > > > >
> > > > >
> > > > > > From the perspective of the new sink api, we might not depend on
> > > > emitting
> > > > >
> > > > > >
> > > > >
> > > > > > records in `NotifyCheckpointComplete`, like using
> > > `OperatorCoordinator`
> > > > >
> > > > > >
> > > > >
> > > > > > instead.
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > *2. What does the FLIP-147 guarantee?*I think initially this
> FLIP
> > > want
> > > > to
> > > > >
> > > > > >
> > > > >
> > > > > > achieve two targets:
> > > > >
> > > > > >
> > > > >
> >
> > > > > > 1. Tasks/Operators exit correctly (as you mentioned the
> lifecycle of
> > > a
> > > > >
> > > > > >
> > > > >
> > > > > > Task/StreamTask/StreamOperator.).
> > > > >
> > > > > >
> > > > >
> > > > > > 2. Continue to trigger checkpoint after some tasks for mixed
> jobs.
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > I think the first thing is related to the discussion in
> FLINK-21133.
> > > > If I
> > > > >
> > > > > >
> > > > >
> > > > > > understand correctly, in addition to supporting the tasks /
> > > operators
> > > > to
> > > > >
> > > > > >
> > > > >
> > > > > > exit correctly, now we also want to unify the process of the
> tasks
> > > and
> > > > >
> > > > > >
> > > > >
> > > > > > operators for savepoint / finish.
> > > > >
> > > > > >
> > > > >
> > > > > > I think the second thing is orthogonal to the FLINK-21133
> because
> > > there
> > > > >
> > > > > are
> > > > >
> > > > > >
> > > > >
> > > > > > topologies that have both the bounded and unbounded input.
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > *3. How to unify the operator exit process of FLIP-147 with
> > > > >
> > > > > >
> > > > >
> > > > > > stop-with-savepoint?*
> > > > >
> > > > > >
> > > > >
> > > > > > I am not very sure about how to do it yet. But if I understand
> the
> > > > >
> > > > > >
> > > > >
> > > > > > discussion in the jira correctly it needs to introduce some
> logic
> > > into
> > > > >
> > > > > >
> > > > >
> >
> > > > > > `CheckpointCoordinator`, which responses for triggering “the
> unified
> > > > >
> > > > > >
> > > > >
> > > > > > operator exit process”. Am I correct?
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > Best,
> > > > >
> > > > > >
> > > > >
> > > > > > Guowei
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann wrote:
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > Thanks for the explanation Yun and Guowei. I have to admit
> that I
> > > do
> > > > >
> > > > > not
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > fully understand why this is strictly required but I think
> that we
> > > > are
> > > > >
> > > > > >
> > > > >
> > > > > > > touching two very important aspects which might have far
> fetching
> > > > >
> > > > > >
> > > > >
> > > > > > > consequences for how Flink works:
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > 1) Do we want to allow that multiple checkpoints are required
> to
> > > > >
> > > > > >
> > > > >
> > > > > > > materialize results?
> > > > >
> > > > > >
> > > > >
> > > > > > > 2) Do we want to allow to emit records in
> > > notifyCheckpointComplete?
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > For 1) I am not sure whether this has been discussed within
> the
> > > > >
> > > > > community
> > > > >
> > > > > >
> > > > >
> > > > > > > sufficiently. Requiring multiple checkpoints to materialize a
> > > result
> > > > >
> > > > > >
> > > > >
> > > > > > > because of multi level committers has the consequence that we
> > > > increase
> > > > >
> > > > > > the
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > latency from checkpoint interval to #levels * checkpoint
> interval.
> > > > >
> > > > > >
> > > > >
> > > > > > > Moreover, having to drain the pipeline in multiple steps,
> would
> > > break
> > > > >
> > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > stop-with-savepoint --drain because which savepoint do you
> report
> > > to
> > > > >
> > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > user?
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > For 2) allowing to send records after the final
> > > > >
> > > > > notifyCheckpointComplete
> > > > >
> > > > > >
> > > > >
> > > > > > > will effectively mean that we need to shut down a topology in
> > > > multiple
> > > > >
> > > > > >
> > > > >
> > > > > > > steps (in the worst case one operator per checkpoint). This
> would
> > > be
> > > > a
> > > > >
> > > > > >
> > > > >
> > > > > > > strong argument for not allowing this to me. The fact that
> users
> > > can
> > > > >
> > > > > send
> > > > >
> > > > > >
> > > > >
> > > > > > > records after the notifyCheckpointComplete is more by accident
> > > than
> > > > by
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > design. I think we should make this a very deliberate decision
> and
> > > in
> > > > >
> > > > > > doubt
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > I would be in favour of a more restrictive model unless there
> is a
> > > > very
> > > > >
> > > > > >
> > > > >
> > > > > > > good reason why this should be supported.
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > Taking also the discussion in FLINK-21133 [1] into account, it
> > > seems
> > > > to
> > > > >
> > > > > > me
> > > > >
> > > > > >
> > > > >
> > > > > > > that we haven't really understood what kind of guarantees we
> want
> > > to
> > > > >
> > > > > give
> > > > >
> > > > > >
> > > > >
> > > > > > > to our users and how the final checkpoint should exactly work.
> I
> > > > >
> > > > > > understand
> > > > >
> > > > > >
> > > > >
> > > > > > > that this is not included in the first scope of FLIP-147 but I
> > > think
> > > > >
> > > > > this
> > > > >
> > > > > >
> > > > >
> > > > > > > is so important that we should figure this out asap. Also
> because
> > > the
> > > > >
> > > > > > exact
> > > > >
> > > > > >
> > > > >
> > > > > > > shut down behaviour will have to be aligned with the lifecycle
> of
> > > a
> > > > >
> > > > > >
> > > > >
> > > > > > > Task/StreamTask/StreamOperator. And last but not least because
> > > other
> > > > >
> > > > > >
> > > > >
> > > > > > > features such as the new sink API start building upon a shut
> down
> > > > model
> > > > >
> > > > > >
> > > > >
> > > > > > > which has not been fully understood/agreed upon.
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > [1] https://issues.apache.org/jira/browse/FLINK-21133
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > Cheers,
> > > > >
> > > > > >
> > > > >
> > > > > > > Till
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma wrote:
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > Thanks Yun for the detailed explanation.
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > A simple supplementary explanation about the sink case:
> Maybe we
> > > > >
> > > > > could
> > > > >
> > > > > >
> > > > >
> > > > > > > use
> > > > >
> > > > > >
> > > > >
> > > > > > > > `OperatorCoordinator` to avoid sending the element to the
> > > > downstream
> > > > >
> > > > > >
> > > > >
> > > > > > > > operator.
> > > > >
> > > > > >
> > > > >
> > > > > > > > But I agree we could not limit the users not to emit records
> in
> > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > `notiyCheckpointComplete`.
> > > > >
> > > > > >
> > > > >
> > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > Best,
> > > > >
> > > > > >
> > > > >
> > > > > > > > Guowei
> > > > >
> > > > > >
> > > > >
> > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao
> > > > >
> > > > > >
> > > > >
> > > > > > > > wrote:
> > > > >
> > > > > >
> > > > >
> > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Hi all,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > I'd like to first detail the issue with emitting records
> in
> > > > >
> > > > > >
> > > > >
> > > > > > > > > notifyCheckpointComplete for context. For specific usage,
> > > > >
> > > > > >
> > > > >
> > > > > > > > > an example would be for sink, it might want to write some
> > > > metadata
> > > > >
> > > > > >
> > > > >
> > > > > > > after
> > > > >
> > > > > >
> > > > >
> > > > > > > > > all the transactions are committed
> > > > >
> > > > > >
> > > > >
> > > > > > > > > (like write a marker file _SUCCESS to the output
> directory).
> > > This
> > > > >
> > > > > > case
> > > > >
> > > > > >
> > > > >
> > > > > > > is
> > > > >
> > > > > >
> > > > >
> > > > > > > > > currently supported via the two level
> > > > >
> > > > > >
> > > > >
> > > > > > > > > committers of the new sink API: when received
> endOfInput(),
> > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > Committer
> > > > >
> > > > > >
> > > > >
> > > > > > > > > wait for another checkpoint to
> > > > >
> > > > > >
> > > > >
> > > > > > > > > commits all the pending transactions and emit the list of
> > > files
> > > > to
> > > > >
> > > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > > GlobalCommitter. The GlobalCommitter
> > > > >
> > > > > >
> > > > >
> > > > > > > > > would wait for another checkpoint to also write the
> metadata
> > > with
> > > > >
> > > > > 2pc
> > > > >
> > > > > >
> > > > >
> > > > > > > > > (Although sometimes 2pc is not needed
> > > > >
> > > > > >
> > > > >
> > > > > > > > > for writing metadata, it should be only an optimization
> and
> > > still
> > > > >
> > > > > >
> > > > >
> > > > > > > > requires
> > > > >
> > > > > >
> > > > >
> > > > > > > > > the Committer do commit before
> > > > >
> > > > > >
> > > > >
> > > > > > > > > notifying the global Committer. Also another note is
> > > > >
> > > > > GlobalCommitter
> > > > >
> > > > > > is
> > > > >
> > > > > >
> > > > >
> > > > > > > > > also added for some other cases
> > > > >
> > > > > >
> > > > >
> > > > > > > > > like some sinks want an commiter with dop = 1, like
> > > IceBergSink).
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > However, a more general issue to me is that currently we
> do
> > > not
> > > > >
> > > > > limit
> > > > >
> > > > > >
> > > > >
> > > > > > > > > users to not emit records in
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > notifyCheckpointComplete in the API level. The sink case
> could
> > > be
> > > > >
> > > > > >
> > > > >
> > > > > > > viewed
> > > > >
> > > > > >
> > > > >
> > > > > > > > > as a special case, but in addition
> > > > >
> > > > > >
> > > > >
> > > > > > > > > to this one, logically users could also implement their
> own
> > > cases
> > > > >
> > > > > > that
> > > > >
> > > > > >
> > > > >
> > > > > > > > > emits records in notifyCheckpointComplete.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Best,
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Yun
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > ------------------Original Mail ------------------
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Sender:Arvid Heise
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Send Date:Fri Feb 12 20:46:04 2021
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Recipients:dev
> > > > >
> > > > > >
> > > > >
> > > > > > > > > CC:Yun Gao
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After
> > > Tasks
> > > > >
> > > > > >
> > > > >
> > > > > > > Finished
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Hi Piotr,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Thank you for raising your concern. Unfortunately, I do
> not
> > > have
> > > > a
> > > > >
> > > > > >
> > > > >
> > > > > > > better
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > idea than doing closing of operators intermittently with
> > > > >
> > > > > checkpoints
> > > > >
> > > > > > (=
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > multiple last checkpoints).
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > However, two ideas on how to improve the overall user
> > > experience:
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > 1. If an operator is not relying on
> notifyCheckpointComplete,
> > > we
> > > > >
> > > > > can
> > > > >
> > > > > >
> > > > >
> > > > > > > > close
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > it faster (without waiting for a checkpoint). In general,
> I'd
> > > > >
> > > > > assume
> > > > >
> > > > > >
> > > > >
> > > > > > > that
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > almost all non-sinks behave that way.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > 2. We may increase the checkpointing frequency for the
> last
> > > > >
> > > > > >
> > > > >
> > > > > > > checkpoints.
> > > > >
> > > > > >
> > > > >
> > > > > > > > We
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > need to avoid overloading checkpoint storages and task
> > > managers,
> > > > >
> > > > > but
> > > > >
> > > > > > I
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > assume the more operators are closed, the lower the
> > > checkpointing
> > > > >
> > > > > >
> > > > >
> > > > > > > > interval
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > can be.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > For 1, I'd propose to add (name TBD):
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > default boolean StreamOperator#requiresFinalCheckpoint() {
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > return true;
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > }
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > This means all operators are conservatively (=slowly)
> closed.
> > > For
> > > > >
> > > > > > most
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > operators, we can then define their behavior by overriding
> in
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > AbstractUdfStreamOperator
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > @Override
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > boolean
> AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > return userFunction instanceof CheckpointListener;
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > }
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > This idea can be further refined in also adding
> > > > >
> > > > > > requiresFinalCheckpoint
> > > > >
> > > > > >
> > > > >
> > > > > > > > to
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > CheckpointListener to exclude all operators with UDFs that
> > > > >
> > > > > implement
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > CheckpointListener but do not need it for 2pc.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > @Override
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > boolean
> AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > return userFunction instanceof CheckpointListener &&
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > ((CheckpointListener)
> userFunction).requiresFinalCheckpoint();
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > }
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > That approach would also work for statebackends/snapshot
> > > > strategies
> > > > >
> > > > > >
> > > > >
> > > > > > > that
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > require some 2pc.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > If we can contain it to the @PublicEvolving
> StreamOperator, it
> > > > >
> > > > > would
> > > > >
> > > > > > be
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > better of course.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Best,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Arvid
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > wrote:
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > Hey,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > I would like to raise a concern about implementation of
> the
> > > > final
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > checkpoints taking into account operators/functions that
> are
> > > > >
> > > > > >
> > > > >
> > > > > > > > implementing
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > two phase commit (2pc) protocol for exactly-once
> processing
> > > > with
> > > > >
> > > > > > some
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > external state (kept outside of the Flink). Primarily
> > > > >
> > > > > exactly-once
> > > > >
> > > > > >
> > > > >
> > > > > > > > sinks.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > First of all, as I understand it, this is not planned in
> the
> > > > >
> > > > > first
> > > > >
> > > > > >
> > > > >
> > > > > > > > > version
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > of this FLIP. I'm fine with that, however I would
> strongly
> > > > >
> > > > > > emphasize
> > > > >
> > > > > >
> > > > >
> > > > > > > > this
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > in every place we will be mentioning FLIP-147 efforts.
> This
> > > is
> > > > >
> > > > > >
> > > > >
> > > > > > > because
> > > > >
> > > > > >
> > > > >
> > > > > > > > > me,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > as a user, upon hearing "Flink supports checkpointing
> with
> > > > >
> > > > > bounded
> > > > >
> > > > > >
> > > > >
> > > > > > > > > inputs"
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > I would expect 2pc to work properly and to commit the
> > > external
> > > > >
> > > > > side
> > > > >
> > > > > >
> > > > >
> > > > > > > > > effects
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > upon finishing. As it is now, I (as a user) would be
> > > surprised
> > > > >
> > > > > > with a
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > silent data loss (of not committed trailing data). This
> is
> > > > just a
> > > > >
> > > > > >
> > > > >
> > > > > > > > remark,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > that we need to attach this warning to every blog
> > > > >
> > > > > >
> > > > >
> > > > > > > > post/documentation/user
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > mailing list response related to "Support Checkpoints
> After
> > > > Tasks
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > Finished". Also I would suggest to prioritize the follow
> up
> > > of
> > > > >
> > > > > >
> > > > >
> > > > > > > > supporting
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 2pc.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > Secondly, I think we are missing how difficult and
> > > problematic
> > > > >
> > > > > will
> > > > >
> > > > > >
> > > > >
> > > > > > > be
> > > > >
> > > > > >
> > > > >
> > > > > > > > > 2pc
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > support with the final checkpoint.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > For starters, keep in mind that currently 2pc can be
> > > > implemented
> > > > >
> > > > > by
> > > > >
> > > > > >
> > > > >
> > > > > > > > users
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > using both `@Public` APIs as functions and
> `@PublicEvolving`
> > > > >
> > > > > >
> > > > >
> > > > > > > operators
> > > > >
> > > > > >
> > > > >
> > > > > > > > in
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > any place in the job graph. It's not limited to only the
> > > sinks.
> > > > >
> > > > > For
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > example users could easily implement the `AsynFunction`
> (for
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > `AsyncWaitOperator`) that is using 2pc based on the
> > > > >
> > > > > >
> > > > >
> > > > > > > > `CheckpointListener`
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > interface. I'm not saying it's common, probably just a
> tiny
> > > > >
> > > > > > minority
> > > > >
> > > > > >
> > > > >
> > > > > > > of
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > users are doing that (if any at all), but nevertheless
> > > that's
> > > > >
> > > > > >
> > > > >
> > > > > > > possible
> > > > >
> > > > > >
> > > > >
> > > > > > > > > and
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > currently (implicitly?) supported in Flink.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > Next complication is the support of bounded streams
> > > > >
> > > > > >
> > > > >
> > > > > > > (`BoundedOneInput`
> > > > >
> > > > > >
> > > > >
> > > > > > > > or
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > `BoundedMultiInput` interfaces) and the closing/shutdown
> > > > >
> > > > > procedure
> > > > >
> > > > > > of
> > > > >
> > > > > >
> > > > >
> > > > > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > operators. Currently it works as follows:
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 0. Task receives EndOfPartitionEvent (or source
> finishes)
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > 1. `endOfInput` is called on the first operator in the
> chain
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 2. We quiesce the processing timers
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > >
> (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> > > > for
> > > > >
> > > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > > first
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > operator, so no new timers will be triggered
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 3. We wait for the already fired timers to finish
> executing
> > > > >
> > > > > > (spinning
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > mailbox loop)
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 4. We are closing the first operator
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 5. We go to the next (second) operator in the chain and
> > > repeat
> > > > >
> > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > steps
> > > > >
> > > > > >
> > > > >
> > > > > > > > > 1.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > to 5.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > This is because operators can emit data after processing
> > > > >
> > > > > >
> > > > >
> > > > > > > `endOfInput`,
> > > > >
> > > > > >
> > > > >
> > > > > > > > > from
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > timers, async mailbox actions and inside the `close`
> method
> > > > >
> > > > > itself.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > Now the problem is to support the final checkpoint with
> 2pc,
> > > we
> > > > >
> > > > > > need
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > trigger `snapshotState` and `notifyCheckpointComplete`
> call
> > > at
> > > > >
> > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > very
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > least only after `endOfInput` call on the operator.
> Probably
> > > > the
> > > > >
> > > > > > best
> > > > >
> > > > > >
> > > > >
> > > > > > > > > place
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > would be in between steps 3. and 4. However that means,
> we
> > > > would
> > > > >
> > > > > be
> > > > >
> > > > > >
> > > > >
> > > > > > > > > forced
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > to wait for steps 1. to 3. to finish, then wait for a
> next
> > > > >
> > > > > > checkpoint
> > > > >
> > > > > >
> > > > >
> > > > > > > > to
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > trigger AND complete, before finally closing the head
> > > operator,
> > > > >
> > > > > and
> > > > >
> > > > > >
> > > > >
> > > > > > > > only
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > then we can start closing the next operator in the
> chain:
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 0. Task receives EndOfPartitionEvent (or source
> finishes)
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > 1. `endOfInput` is called on the first operator in the
> chain
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 2. We quiesce the processing timers
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > >
> (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> > > > for
> > > > >
> > > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > > first
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > operator, so no new timers will be triggered
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 3. We wait for the already fired timers to finish
> executing
> > > > >
> > > > > > (spinning
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > mailbox loop)
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > *3b. We wait for one more checkpoint to trigger and for
> the
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > `notifyCheckpointComplete` RPC.*
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 4. We are closing the first operator
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 5. We go to the next (second) operator in the chain and
> > > repeat
> > > > >
> > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > steps
> > > > >
> > > > > >
> > > > >
> > > > > > > > > 1.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > to 5.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > That means, we can close one operator per successful
> > > > checkpoint.
> > > > >
> > > > > To
> > > > >
> > > > > >
> > > > >
> > > > > > > > close
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > 10 operators, we would need 10 successful checkpoints.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > I was thinking about different approaches to this
> problem,
> > > and
> > > > I
> > > > >
> > > > > >
> > > > >
> > > > > > > > couldn't
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > find any viable ones. All I could think of would break
> the
> > > > >
> > > > > current
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > `@Public` API and/or would be ugly/confusing for the
> users.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > For example a relatively simple solution, to introduce a
> > > > >
> > > > > `preClose`
> > > > >
> > > > > >
> > > > >
> > > > > > > or
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > `flush` method to the operators, with a contract that
> after
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > `flush`, operators would be forbidden from emitting more
> > > > records,
> > > > >
> > > > > > so
> > > > >
> > > > > >
> > > > >
> > > > > > > > that
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > we can replace step 4. with this `flush` call, and then
> > > having
> > > > a
> > > > >
> > > > > >
> > > > >
> > > > > > > single
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > checkpoint to finish 2pc for all of the operators inside
> the
> > > > >
> > > > > chain,
> > > > >
> > > > > >
> > > > >
> > > > > > > > > doesn't
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > work. Sheer fact of adding this `flush` method and
> changing
> > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > contract
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > would break the current API and Yun Gao has pointed out
> to
> > > me,
> > > > >
> > > > > that
> > > > >
> > > > > >
> > > > >
> > > > > > > we
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > either already support, or want to support operators
> that
> > > are
> > > > >
> > > > > >
> > > > >
> > > > > > > emitting
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > records from within the `notifyCheckpointComplete` call:
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > Yun Gao:
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > like with the new sink api there might be writer ->
> > > committer
> > > > >
> > > > > ->
> > > > >
> > > > > >
> > > > >
> > > > > > > > global
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > committer, the committer would need to wait for the last
> > > > >
> > > > > checkpoint
> > > > >
> > > > > >
> > > > >
> > > > > > > to
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > commit
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > the last piece of data, and after that it also need to
> > > emit
> > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > list
> > > > >
> > > > > >
> > > > >
> > > > > > > > of
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > transactions get committed to global committer to do
> some
> > > > >
> > > > > >
> > > > >
> > > > > > > finalization
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > logic.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > So it wouldn't solve the problem (at least not fully).
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > I don't know if anyone has any better ideas how to solve
> > > this
> > > > >
> > > > > >
> > > > >
> > > > > > > problem?
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > Piotrek
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > pt., 15 sty 2021 o 14:57 Yun Gao
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > napisał(a):
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > Hi Aljoscha,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > I think so since we seems to do not have other
> divergence
> > > and
> > > > >
> > > > > new
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > objections now. I'll open the vote then. Very thanks!
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > Best,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > Yun
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> ------------------------------------------------------------------
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > From:Aljoscha Krettek
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > Send Time:2021 Jan. 15 (Fri.) 21:24
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > To:dev
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints
> After
> > > > Tasks
> > > > >
> > > > > >
> > > > >
> > > > > > > > > Finished
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > Thanks for the summary! I think we can now move
> towards a
> > > > >
> > > > > [VOTE]
> > > > >
> > > > > >
> > > > >
> > > > > > > > > thread,
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > right?
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > On 2021/01/15 13:43, Yun Gao wrote:
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > >1) For the problem that the "new" root task
> coincidently
> > > > >
> > > > > > finished
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > > >before getting triggered successfully, we have listed
> two
> > > > >
> > > > > > options
> > > > >
> > > > > >
> > > > >
> > > > > > > in
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > >the FLIP-147[1], for the first version, now we are
> not
> > > tend
> > > > to
> > > > >
> > > > > > go
> > > > >
> > > > > >
> > > > >
> > > > > > > > with
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > >the first option that JM would re-compute and
> re-trigger
> > > new
> > > > >
> > > > > >
> > > > >
> > > > > > > sources
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > >when it realized some tasks are not triggered
> > > successfully.
> > > > >
> > > > > This
> > > > >
> > > > > >
> > > > >
> > > > > > > > > option
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > >would avoid the complexity of adding new PRC and
> > > duplicating
> > > > >
> > > > > > task
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > >states, and in average case it would not cause too
> much
> > > > >
> > > > > > overhead.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> >
> > > > > > > > > > > You wrote "we are *not* tend to go with the first
> option",
> > > > but
> > > > >
> > > > > I
> > > > >
> > > > > >
> > > > >
> > > > > > > > think
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > you meant wo write "we tend to *now* go with the first
> > > > option",
> > > > >
> > > > > >
> > > > >
> > > > > > > > right?
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > That's also how it is in the FLIP, I just wanted to
> > > clarify
> > > > for
> > > > >
> > > > > > the
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > > mailing list.
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > >
> > > >
> > >
> > >
> >
> >
> >
>
>

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

Posted by Kezhu Wang <ke...@gmail.com>.
Hi all,

Glad to see convergence here and FLINK-21133:
1. We all prefer single final checkpoint for task not individual
checkpoints for each operators.
2. To above goal, if we have to breaking something, we will.
3. Don’t allow recording emitting in `notifyCheckpointComplete`.

For “end-flushing”, I think both approaches should function in reality, but
we also have options/responsibilities to mitigate effect of breaking
changes:
A. Allowing checkpoint after “close”. Introduce config option to forbid
this during migrating releases.
B. Renaming “close” to “other-end-flushing-method”. We can backport that
newly introducing “end-flushing”(as empty default method) to earlier
releases in following patch releases. The backporting “end-flushing” will
be called just before “close” in future patch releases. We could call
“close” just before “dispose" in future releases and `final` it in
`AbstractStreamOperator` when ready(to break user side code).

If breaking change for this “end-flushing” in inevitable, I am kind of
prefer renaming and backport approach. It is a chance for us to rethink the
whole thing and discard misleading “close”(currently it is mixed/misused
with “end-flushing” and “cleanup-resource” though javadoc claims only
“end-flushing, this could also be considered as a bug though).

Besides this, will FLIP-147 eventually need some ways to decide whether an
operator need final checkpoint @Yun @Guowei ?  @Arvid mentions this in
earlier mail.


For the two phase commit, @Piotrek I like your idea. I think that
“commit-handle” could be return to checkpoint-coordinator through
`notifyCheckpointComplete`. This way that “commit-handle” might be reused
by operator-coordinator’s `notifyCheckpointComplete`. Suppose following
changes:

1. `CompletableFuture<SerializedValue<?>> notifyCheckpointCompleteAsync()`
in operator.
2. `CompletableFuture<Void> notifyCheckpointCompleteAsync(Map<Integer,
CompletableFuture<SerializedValue>> subtasks)` in operator coordinator.

These changes need support from:
* Checkpoint coordinator level to bridge operator and coordinator through
task
* Operator level to compat existing `notifyCheckpointComplete`

The checkpoint procedure will looks like:
1. Trigger checkpoint for operator coordinator.
2. If above succeeds, trigger tasks checkpoint. Abort otherwise.
3. If all above succeeds, complete current checkpoint. Abort otherwise.
4. If job fails after, restore from above “completed” checkpoint.
5. Notify checkpoint completion to tasks.
6. Notify checkpoint completion to coordinators.
7. Wait step#5 and step#6 to succeed. Now it is real completed. Either this
succeed or job failed in meantime ? May be other concurrent conditions.

With these changes, migration FLIP-143 sink to operator coordinator should
be easy.

It will definitely complicate currently complex checkpoint coordinator as
@Till mentioned in FLINK-21133.


Best,
Kezhu Wang

On March 3, 2021 at 01:09:50, Piotr Nowojski (pnowojski@apache.org) wrote:

Hi,

Thanks for reminding me. I think FLIP-147 will have to deal in one way or
another with the (re?)emitting MAX_WATERMARK. There is a pre-existing issue
that watermarks are not checkpointed/stored on state, and there was/is now
clear answer how we should handle this as far as I remember. One
problematic case are two/multiple input tasks or UnionInputGate, where
combined watermark is the min of all inputs (held in memory). The problem
so far is a bit benign, as after recovery we are losing the combined
watermark value, but it's being slowly/lazily restored, as new watermarks
are sent from the sources. With finished sources that won't be a case.

I've spent more time thinking about the two phase commit issue, and I agree
it would be best to have a single final checkpoint for the operators. The
more I think about it, the more I'm leaning toward the
"OperatorCoordinator" approach. Yes, it would need the extra complexity of
running user code on the JobManager, but that's already at least partially
done (FLIP-27). On the other hand it would allow us to keep the runtime
code cleaner and simpler I think. It's still an invasive change.

1. We forbid emitting records from
`notifyCheckpointComplete`/`snapshotState` calls, or at the very least from
the final calls.
2. We need to clarify/clean up contracts for both operators AND user
functions when it comes to `endOfInput()`, `close()` and `dispose()`
methods. For example:
a) we keep `endOfInput()` on the operators level and add an equivalent of
it to the user functions
b) we move the "flushing buffered records" part of the `close()` contract
to `endOfInput()`
c) we remove `close()` methods altogether (IMO it's better to have an
explicit false conflict, rather than a silent real one)
d) we keep `dispose()` on the operator level for release resources, and
we add such method to the user functions
3. We change (or add another version) a return type of either `void
snapshotState(...)` or `void notifyCheckpointComplete(...)` calls, so that
they will return some kind of `GlobalCommitHandle`.

Especially this point 2. is intrusive. Now, putting all of that together.
When a task receives a logical "end of input" signal (current
`EndOfPartitionEvent`), it begins shutting down procedure (let's assume we
create `GlobalCommitHandle` in `snapshotState`).

0. n = 1
1. endOfInput on the n'th operator
2. quisec timers and mailbox executor for n'th operator
// after this point n'th operator is forbidden to emit any new records
3. n += 1, and go to 1., until we handle all of the operators
4. wait for final checkpoint, issue `snapshotState` and collect
`GlobalCommitHandle`s and send them to the JM
5. JM/CheckpointCoordinator collects all `GlobalCommitHandles`. They are an
integral part of the checkpoint. They could be added to the respective
`OperatorCoordinator`s via `handleEventFromOperator` as an `OperatorEvent`.
6. When checkpoint is considered completed, `notifyCheckpointComplete` are
being issued, both on the operators, and `OperatorCoordinator` - local
commit and global commits would be happening at the same time, or even
global commit code would be executed before local final commit in this
version.
7. Only now can we finally dispose of all operators in the task.

If there is a failure, especially before we manage to perform all
`notifyCheckpointComplete` calls (for example on the
`OperatorCoordinator`), we would need to recover from last "completed"
checkpoint, recover attached `GlobalCommitHandle`s , and re-commit them. It
also means we need to recover already finished operators, that will never
process any records, just to issue the final `notifyCheckpointComplete`
call and make sure they commit their external side effects.

There are at least a couple of questions about this general idea:
I. Naming of the methods (I suggested to drop `close()` and keep in the
operators and introduce in the functions `endOfInput()` and `dispose()`
II. When should the `GlobalCommitHandle` be created? Should it be returned
from `snapshotState()`, `notifyCheckpointComplete()` or somewhere else?
III. What should be the ordering guarantee between global commit and local
commit, if any? Actually the easiest to implement would be undefined, but
de facto global commit happening before local commits (first invoke
`notifyCheckpointComplete()` on the `OperatorCoordinator` and either after
or in parallel send `notifyCheckpointComplete()` RPCs). As far as I can
tell, undefined order should work for the use cases that I'm aware of.
IV. Can we reuse `OperatorCoordinator` abstraction for this purpose?
Currently it's being used to induce checkpoints with FLIP-27 sources as far
as I understand, here we would be using it for different purposes.

Best,
Piotrek

pon., 1 mar 2021 o 07:30 Yun Gao <yu...@aliyun.com> napisał(a):

> (Sorry that I repeat this mail since the last one is not added into the
> same mail list thread,
> very sorry for the inconvenience)
>
> Hi all,
>
> Very thanks for all the deep thoughts!
>
> > How to implement the stop-with-savepoint --drain/terminate command with
> > this model: One idea could be to tell the sources that they should stop
> > reading. This should trigger the EndOfPartitionEvent to be sent
> > downstream.
> > This will transition all operators into the TERMINATING state.
> >
> > Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
> > returned. To achieve above, possible works should be required:
> > * Promote `EndOfPartitionEvent` from `Task` to `StreamTask`. This may
> > have some interferences with BatchTask or network io stack.
> > * Or introducing stream task level `EndOfUserRecordsEvent`(from
PR#14831
> > @Yun @Piotr)
> > * Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.
>
> I also have similar concern with Kezhu for the issue whether do we need
to
> introduce a new message
> to notify the operators to endOfInput/close ? The main concerns of
reusing
> that EndOfPartitionEvent
> is that
> 1. The EndOfPartitionEvent is currently emitted in Task instead of
> StreamTask, we would need some
> refactors here.
> 2. Currently the InputGate/InputChannel would be released after the
> downstream tasks have received
> EndOfPartitionEvent from all the input channels, this would makes the
> following checkpoint unable to
> perform since we could not emit barriers to downstream tasks ?
>
> Regarding the MAX_WATERMARK, I still not fully understand the issue since
> it seems to me
> that now Flink won't snapshot the watermark now? If the job failover, the
> window operator
> would reload all the pending windows before flushed by MAX_WATERMARK and
> when the
> job finish again, it would re-emit the MAX_WATERMARK?
>
> Best,
> Yun
>
>
> ------------------------------------------------------------------
> From:Kezhu Wang <ke...@gmail.com>
> Send Time:2021 Mar. 1 (Mon.) 01:26
> To:Till Rohrmann <tr...@apache.org>
> Cc:Piotr Nowojski <pi...@gmail.com>; Guowei Ma <
> guowei.mgw@gmail.com>; dev <de...@flink.apache.org>; Yun Gao <
> yungao.gy@aliyun.com>; jingsonglee0@gmail.com <ji...@gmail.com>
> Subject:Re: Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> Finished
>
> In “stop-with-savepoint —drain”, MAX_WATERMARK is not an issue. For
normal
> finishing task, not allowing unaligned checkpoint does not solve the
> problem as MAX_WATERMARK could be persisted in downstream task. When
> scenario @Piotr depicted occurs, downstream(or further downstream) window
> operator will count all inputs as late.
>
> > If we ensure that the MAX_WATERMARK is only persisted in state if a
> recovery will trigger immediately the shut down of this operator, then it
> shouldn't be an issue.
>
> You are right in case the assumption holds, I have same thought as you
> before. But I am kind of worry about whether it is too prefect to be
> fragile. This requires strong guarantee from implementation that a
recovery
> from TERMINATING stage should go directly to that stage.
>
> > I think the MAX_WATERMARK event should be sent either just before or
with
> the EndOfPartitionEvent.
>
> I prefer “with the EndOfPartitionEvent”. EndOfPartitionEvent itself
already
> carry what ending MAX_WATERMARK try to express. May be we can reuse it ?
> @Piotr
>
> A preview work[1] from @Yun in PR#14831 explains EndOfPartitionEvent as
> checkpoint barrier if there are pending checkpoints.
>
>
> [1]:
>
>
https://github.com/gaoyunhaii/flink/commit/1aaa80fb0afad13a314b06783c61c01b9414f91b#diff-d4568b34060bc589cd1354bd125c35aca993dd0c9fe9e4460dfed73501116459R177
>
>
> Best,
> Kezhu Wang
>
> On February 28, 2021 at 21:23:31, Till Rohrmann (trohrmann@apache.org)
> wrote:
>
> I think you are right with the problem of endOfInput. endOfInput should
not
> be used to commit final results. In fact if this termination fails then
we
> might end up in a different outcome of the job which is equally valid as
> the one before the failure.
>
> Concerning unaligned checkpoints, I think they don't play well together
> with draining a streaming pipeline. The problem is that in the draining
> case you want to process all records which are still in flight but
> unaligned checkpoints don't guarantee this as they can jump in flight
> records.
>
> I think the MAX_WATERMARK event should be sent either just before or with
> the EndOfPartitionEvent. If we ensure that the MAX_WATERMARK is only
> persisted in state if a recovery will trigger immediately the shut down
of
> this operator, then it shouldn't be an issue.
>
> Cheers,
> Till
>
> On Sun, Feb 28, 2021 at 9:06 AM Kezhu Wang <ke...@gmail.com> wrote:
>
> > Hi Till,
> >
> > Just for bookkeeping, some observations from current implementation.
> >
> > > With this model, the final checkpoint is quite simple because it is
> > ingrained in the lifecycle of an operator. Differently said an operator
> > will only terminate after it has committed its side effects and seen
the
> > notifyCheckpointComplete message (if it is stateful).
> >
>
> > Currently, we could not mark this operator(or subtask) as terminated
since
> > result of `notifyCheckpointComplete`(possible side effect committing)
is
> > not taken into account of the belonging checkpoint. The job has to run
to
> > next safe point(finished or next checkpoint success) to be marked as
> > “terminated”.
> >
> > > How to implement the stop-with-savepoint --drain/terminate command
with
> > this model: One idea could be to tell the sources that they should stop
> > reading. This should trigger the EndOfPartitionEvent to be sent
> > downstream.
> > This will transition all operators into the TERMINATING state.
> >
> > Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
> > returned. To achieve above, possible works should be required:
> > * Promote `EndOfPartitionEvent` from `Task` to `StreamTask`. This may
> > have some interferences with BatchTask or network io stack.
> > * Or introducing stream task level `EndOfUserRecordsEvent`(from
PR#14831
> > @Yun @Piotr)
> > * Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.
> >
> > Besides this, I would like to quote some discussion from FLINK-21467
> > between @Piotr and me:
> >
> > From @Piotr
> > > Note, that it's not only that endOfInput can be called multiple
times.
>
> > There is a very remote possibility that the following scenario will
happen:
> > 1. checkpoint is taken (successfully)
> > 2. sources are finishing
> > 3. endOfInput is issued
> > 4. job fails
> > 5. job restarts to checkpoint 1.
> > 6. after failover, because of some non deterministic logic in the
source,
> > sources are not finishing
> >
> > From me
> > > But I think there is little work Flink can do to cope with this kind
of
> > issues. The checkpoint could be a savepoint triggered from user side
and
> > the "non deterministic logic" could be a change from user(eg. changing
of
> > stoppingOffsets in KafkaSource).
> >
>
> > > I think the "non deterministic logic" could cause trouble in
combination
> > with unaligned checkpoint and downstream window operator. Unaligned
> > checkpoint will persist "MAX_WATERMARK" in state, after restarting,
> > "MAX_WATERMARK" will cause downstream window operator ignores all
future
> > inputs.
> >
> > FLIP-147 demands no new records from end-of-stream-flushing, but source
> > will emit “MAX_WATERMARK” on ending. Previously, I thought it is not a
>
> > valid issue, but turn out that it could cause trouble under scenario
listed
> > by @Piotr if I am not wrong.
> >
> >
> > PR#14831: https://github.com/apache/flink/pull/14831
> > FLINK-21467: https://issues.apache.org/jira/browse/FLINK-21467
> >
> >
> > Best,
> > Kezhu Wang
> >
> > On February 27, 2021 at 18:12:20, Till Rohrmann (trohrmann@apache.org)
> > wrote:
> >
> > Thanks for all your thoughts. I think we should further think through
>
> > whether to allow checkpoints after an operator has emitted all its
records
> > (e.g. after close is called currently) or not. I think by doing this we
> > would nicely decouple the checkpoint taking from the operator lifecycle
> > and
> > wouldn't need special checkpoints/savepoints for the final checkpoint
and
> > stop-with-savepoint --drain. Let me try to explain this a bit more
> > detailed.
> >
> > If we say an operator has the rough lifecycle RUNNING => TERMINATING =>
> > TERMINATED where we go from RUNNING into TERMINATING after we have seen
> > the
> > EndOfPartitionEvent and flushed all our records. The operator goes from
> > TERMINATING => TERMINATED if it has persisted all its possible side
> > effects. Throughout all states, it is possible to trigger a checkpoint.
A
> > stateless operator will immediately go from TERMINATING to TERMINATED
> > whereas a stateful operator would wait for another checkpoint to be
> > triggered and successfully completed (notifyCheckpointComplete).
> >
> > With this model, the final checkpoint is quite simple because it is
> > ingrained in the lifecycle of an operator. Differently said an operator
> > will only terminate after it has committed its side effects and seen
the
>
> > notifyCheckpointComplete message (if it is stateful). Here it is
important
> > to note that in the streaming case, different bounded operators can
> > terminate at different times. They don't have to terminate all with the
> > same checkpoint.
> >
> > How to implement the stop-with-savepoint --drain/terminate command with
> > this model: One idea could be to tell the sources that they should stop
> > reading. This should trigger the EndOfPartitionEvent to be sent
> > downstream.
>
> > This will transition all operators into the TERMINATING state. Next the
JM
> > can trigger a checkpoint to shut the operators down and not to wait for
> > the
> > next automatic checkpoint trigger event.
> >
> > By allowing checkpoints throughout the entire lifecycle of an operator
we
> > disallow sending records from notifyCheckpointComplete because this
> > message
> > will also be sent in the state TERMINATING where an operator has
already
> > produced all of its records.
> >
> > What do you think? Maybe this model overlooks some important cases. One
> > downside is that we will break the operator API with changing the
> > lifecycle
> > of an operator.
> >
> > Cheers,
> > Till
> >
> >
> >
> > On Fri, Feb 26, 2021 at 7:29 AM Kezhu Wang <ke...@gmail.com> wrote:
> >
> > > Hi Yun,
> > >
> > > The termination phase I drafted depends on a de-fact that
>
> > > `notifyCheckpointComplete` is ignored currently after `close`. Thus,
any
> > > external materialization inside `close`(including `endInput`) or
> > > `notifyCheckpointComplete` is either not guaranteed or not committed.
> > >
> > > I tried to emphasize no-breaking changes and post-pone migration in
> > later
> > > releases for this termination phase. But now, I am kind of worry
about
> > > whether it will cause long-term maintenance hell.
> > >
> > > Personally, I think allowing checkpoint after `close`(@Till proposed
> > this
> > > in FLINK-21133) could be the minimal change. But there are concerns:
> > > * It will break some existing code possibly in a silent way.(@Piotr
> > pointed
> > > this already in FLINK-21133)
> > > * I think the name `close` is kind of misleading. (@Piotr suggested
> > > renaming this to `finish` to not break code silently in FLINK-21133)
> > >
> > > > stop-with-savepoint --drain would wait for a specific savepoint
> > >
> > > For stop-with-savepoint, the checkpoint barrier is already created or
> > > received there.
> > >
> > >
> > > Best,
> > > Kezhu Wang
> > >
> > > On February 25, 2021 at 21:02:30, Yun Gao (yungao.gy@aliyun.com
> ) wrote:
> > >
> > > Hi all,
> > >
> > > Very thanks for the discussions!
> > >
> > >
> > >
> > > A. Regarding how to avoid emitting records in
notifyCheckpointComplete:
> > >
> > > Currently the structure of a new sink is writer -> committer ->
global
> > > committer and the paralellism of global committer
> > > must be one. By design it would be used in several cases:
> > > 1. writer -> committer: for normal sinks that write pending data in
> > writer
> > > and commit pending data in committer.
> > > 2. writer -> global committer: for sinks require the committer's
> > > parallelism be 1, like iceberg sink which
> > > relies on optimistic lock to commit data hopes to reduce the
conflicts.
> > > 3. writer -> committer -> global committer: for sinks that also need
to
> > > write meta data (like _SUCCESS file
> > > or add record in hive metastore) in global committer.
> > >
> > > The third case would cause the cascade commit problem. To overcome
this
> > > problem, we would like to
> > > 1. Previously the global committer also support writing metadata with
> > 2pc.
> > > Now we disable this mode
> > > and always rely on the property that writing metadata is repeatable.
> > With
> > > this limitation there should no
> > > need of cascade commit, but the committer would still emit records in
> > > notifyCheckpointComplete.
> > > 2. We further move global committer in the case 3 to the operator
> > > coordinator. Then the committer would
> > > not need to emit records, but communicates with its operator
> > coordinator.
> > >
> > > One core issue for using the OperatorCoordinator is how to keep the
> > > communication between the operator
> > > and the OperatorCoordinator exactly-once. Since the message is always
> > from
> > > the oeprator to the OperatorCoordinator
> > > in this case, we would only need to bookkeep the message sent between
> > the
> > > OperatorCoordinator takes snapshot and
> > > the Operator takes snapshot in the state of the Operator.
> > >
> > > On how to achieve the change in detail we would still need some
think,
> > it
> > > currently seems we would have to had
> > > some modification to the current new sink api.
> > >
> > >
> > >
> > > B. Regarding the stop-with-savepoint --drain
> > >
> > > Very thanks @Piotr for the further explanation and now I realize I
have
> > > understand wrongly for the semantics of
> > > stop-with-savepoint --drain. Now I think that the problem should be
we
> > > should also include the records produced in
> > > `endOfInput()` and `close()` also in the last savepoint, am I
correct?
> > If
> > > so, it seems we still have some undetermined options for
> > > the lifecycle of the operator, like in Kezhu's proposal the close()
> > happens
> > > at last, but it seems close() might also emit records (
> > > so now the operator are closed with op1's close() -> op2's
endOfInput()
> > ->
> > > op2's close() -> op3's endOfinput -> ...) ?
> > >
> > > And on the other side, as Kezhu has also proposed, perhapse we might
> > have
> > > the stop-with-savepoint --drain and normal exit in the same process,
> > > but have slightly difference in that stop-with-savepoint --drain
would
> > wait
> > > for a specific savepoint and in normal exit, the operator
> > > might wait for arbitrary checkpoint. If we could achieve not emitting
> > > records in notifyCheckpointComplete, stop-with-savepoint --drain
could
> > > be done with one savepoint, and for the normal exit, the operator
would
> > not
> > > need to wait for other slow operators to exit.
> > >
> > > Best,
> > > Yun
> > >
> > >
> > >
> > > ------------------Original Mail ------------------
> > > *Sender:*Kezhu Wang <ke...@gmail.com>
> > > *Send Date:*Thu Feb 25 15:11:53 2021
> > > *Recipients:*Flink Dev <de...@flink.apache.org>, Piotr Nowojski <
> > > piotr.nowojski@gmail.com>
> > > *CC:*Guowei Ma <gu...@gmail.com>, jingsonglee0@gmail.com <
> > > jingsonglee0@gmail.com>
> > > *Subject:*Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > > Finished
> > >
> > > > Hi all, thanks for driving this and especially Piotr for re-active
> > this
> > > >
> > > > thread.
> > > >
> > > >
> > > >
> > > > First, for `notifyCheckpointComplete`, I have strong preference
> > towards
> > > >
> > > > "shut down the dataflow
> > > >
> > > > pipeline with one checkpoint in total", so I tend to option
dropping
> > > "send
> > > >
> > > > records" from
> > > >
> > > > `notifyCheckpointComplete` for next level committing in pipeline,
if
> > we
> > > >
> > > > ever support it. Without this,
> > > >
> > > > we are unable to stop a pipeline manually with all results
> > materialized.
> > > >
> > > >
> > > >
> > > > Second, for shutdown unification of `stop-with-savepoint --drain`
and
> > > >
> > > > FLIP-147, I draft following
> > > >
> > > > phase based on emerging proposals(mainly by Piotr and Till) in this
> > > thread
> > > >
> > > > and FLINK-21133.
> > > >
> > > >
> > > >
> > > > ```java
> > > >
> > > > // StreamTask termination phase
> > > >
> > > > finish()(Call StreamOperator.finish in chaining order)
> > > >
>
> > > > advanceToEndOfEventTime()(nop if for no-source inputs, this could
also
> > be
> > > >
> > > > done in input processor or finish ?)
> > > >
> > > > if (there-is-a-pending-terminate-savepoint) {
> > > >
> > > > triggerBarrierForDownStream();
> > > >
> > > > waitCheckpointComplete();
> > > >
> > > > } else if (require-two-phase-commit-in-shutdown) {
> > > >
> > > > waitFinalCheckpoint();
> > > >
> > > > waitCheckpointComplete();
> > > >
> > > > }
> > > >
> > > > // TODO: Ignore/Cancel all checkpoints from this point(or hit to
> > > >
> > > > coordinator in final checkpoint response ?).
> > > >
> > > > close();
> > > >
> > > > ```
> > > >
> > > >
> > > >
> > > > Branches in `if` could reside in different code paths, but the
effect
> > > >
> > > > should be similar to above.
> > > >
> > > > The divergence reflects fact that "stop-with-savepoint --drain" is
> > > >
> > > > triggered already while we need
> > > >
> > > > to wait for final checkpoint in natural input exhausting.
> > > >
> > > >
> > > >
> > > > With carefully chosen default functions, we will not break existing
> > > >
> > > > interfaces.
> > > >
> > > > * `StreamOperator.finish`
> > > >
> > > > * `RichFunction.finish`
> > > >
> > > >
> > > >
> > > > For existing codes:
> > > >
> > > > 1. "stop-with-savepoint" with no migration, it should behaves as
> > before.
> > > >
> > > > 2. "require-two-phase-commit-in-shutdown" evaluates to false and no
> > > >
> > > > migration, it is same as before.
> > > >
> > > > 3. "require-two-phase-commit-in-shutdown" evaluates to true and no
> > > >
> > > > migration, records in `close` will
> > > >
> > > > lose.
> > > >
> > > >
> > > >
> > > > For case#3:
> > > >
> > > > A. No external materialization in `StreamOperator.close`: Nothing
to
> > > lose.
> > > >
> > > > B. External materialization only in `StreamOperator.close`: I don't
> > think
> > > >
> > > > Flink ever claimed that there is
> > > >
> > > > any guarantee for this situation.
> > > >
> > > > C. External materialization in `notifyCheckpointComplete` from
> > > >
> > > > `StreamOperator.close`: But due to fact that
> > > >
> > > > `notifyCheckpointComplete` was ignored after operator
> > > >
> > > > closed(FLINK-16383), so there will be no external
> > > >
> > > > materialization to lose.
> > > >
> > > >
> > > >
> > > > Then, we could recommend users to migrate possible
> > > “end-of-stream-flushing”
> > > >
> > > > from “close" to “finish”.
> > > >
> > > >
> > > >
> > > > Best,
> > > >
> > > > Kezhu Wang
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > On February 24, 2021 at 23:47:36, Piotr Nowojski (
> > > piotr.nowojski@gmail.com
> > > > )
> > > >
> > > > wrote:
> > > >
> > > >
> > > >
> > > > Thanks for the reponses Guowei and Yun,
> > > >
> > > >
> > > >
> > > > Could you elaborate more/remind me, what does it mean to replace
> > emitting
> > > >
>
> > > > results from the `notifyCheckpointComplete` with
`OperatorCoordinator`
> > > >
> > > > approach?
> > > >
> > > >
> > > >
> > > > About the discussion in FLINK-21133 and how it relates to FLIP-147.
> > You
> > > are
> > > >
>
> > > > right Yun gao, that in case of `stop-with-savepoint --drain` the
whole
> > > job
> > > >
> > > > finishes, while in FLIP-147 some of the sources can work for
> > arbitrarily
> > > >
>
> > > > long time after others have already finished. However from the
runtime
> > > >
> > > > perspective, especially how to close a Task and it's operators,
there
> > is
> > > no
> > > >
> > > > difference between those two cases. In both cases, we need to end
> > input,
> > > >
>
> > > > shut down processing time timers and commit external side effects
(two
> > > >
> > > > phase commit support) before exiting the task. That's how the
> > discussion
> > > >
> > > > about the behaviour of "stop-with-savepoint" was connected with
> > FLIP-147.
> > > >
> > > >
> > > >
> > > > Currently on master, "stop-with-savepoint --drain" drains/flushes
> > > buffered
> > > >
>
> > > > records and deals correctly with timers, but all of that happens
AFTER
> > > >
> > > > savepoint was completed. So any records flushed from the operators
> > during
> > > >
> > > > endOfInput/close/shutting down processing timers are never
committed
> > to
> > > >
> > > > external systems. This is exactly the same problem as the "two
phase
> > > >
>
> > > > commit" problem of FLIP-147, that should have the same solution and
it
> > > >
>
> > > > should be solved at the same time. For example if we go with the
*very
> > > >
> > > > undesirable* "one closed operator per one completed
> > > checkpoint/savepoint",
> > > >
> > > > in both cases CheckpointCoordinator, Scheduler and Task would need
to
> > > keep
> > > >
> > > > the task alive and keep triggering checkpoints for that task, until
> > all
> > > >
> > > > operators in the operator chain are closed (one closed operator per
> > one
> > > >
> > > > completed checkpoint).
> > > >
> > > >
> > > >
> > > > Piotrek
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > śr., 24 lut 2021 o 11:30 Yun Gao napisał(a):
> > > >
> > > >
> > > >
> > > > > Hi Till, Guowei,
> > > >
> > > > >
> > > >
> > > > > Very thanks for initiating the disucssion and the deep thoughts!
> > > >
> > > > >
> > > >
>
> > > > > For the notifyCheckpointComplete, I also agree we could try to
avoid
> > > >
> > > > > emitting new records in notifyCheckpointComplete via using
> > > >
> > > > > OperatorCoordinator
> > > >
> > > > > for new sink API. Besides, the hive sink might also need some
> > > >
> > > > modification
> > > >
> > > > > for it also emits records in notifyCheckpointComplete.
> > > >
> > > > >
> > > >
>
> > > > > For unifying the process of stopping with savepoint and finished
due
> > to
> > > >
> > > > > all records
> > > >
> > > > > are processed, I also agree with that unifying would always be
> > better
> > > if
> > > >
> > > > > we could achieve,
> > > >
> > > > > but I'm still not fully catch up with the implementation: Based
on
> > the
> > > >
> > > > > discussion in FLINK-21133,
> > > >
>
> > > > > my understanding is that for stopping with savepoint, now we want
to
> > > >
> > > > first
> > > >
> > > > > stop the source, then we
> > > >
> > > > > trigger a savepoint, and after the source received
> > > >
> > > > > notifyCheckpointComplete, the source would
> > > >
>
> > > > > start emitting EndOfPartitionEvent to finish the job, am I
correct ?
> > > >
> > > > >
> > > >
> > > > > For normal finish, a difference to me might be if we have
multiple
> > > >
> > > > > sources, we could not guarantee
> > > >
> > > > > when the sources are to finish. We might have one source run one
1
> > > minute
> > > >
> > > > > and another one run for
> > > >
>
> > > > > 1 hour. To unify with the process with stop with savepoint, we
might
> > > need
> > > >
> > > > > to hold the fast source until
> > > >
> > > > > all the sources are finished? An coordinator would be introduced
to
> > > count
> > > >
> > > > > the number of sources
> > > >
>
> > > > > runing and trigger the final savepoint / checkpoint. For the
extreme
> > > >
> > > > > cases, if we have both bounded and
> > > >
> > > > > unbounded sources, we might only count how much bounded source
are
> > > >
> > > > > remaining ? And if all the bounded
> > > >
> > > > > sources are finished we would trigger the special checkpoint.
After
> > all
> > > >
> > > > > the bounded part of the graph are
> > > >
> > > > > finished, the the remaining part could still do checkpoint and
> > commit
> > > >
> > > > data
> > > >
> > > > > with FLIP-147.
> > > >
> > > > >
> > > >
> > > > > Best,
> > > >
> > > > > Yun
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > ------------------Original Mail ------------------
> > > >
> > > > > Sender:Guowei Ma
> > > >
> > > > > Send Date:Wed Feb 24 17:35:36 2021
> > > >
> > > > > Recipients:dev
> > > >
> > > > > CC:Arvid Heise
> > > >
> > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > > Finished
> > > >
> > > > > Hi, Till
> > > >
> > > > >
> > > >
> > > > > Thank you very much for your careful consideration
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > *1. Emit records in `NotifyCheckpointComplete`.*
> > > >
> > > > >
> > > >
> > > > > Sorry for making you misunderstanding because of my expression. I
> > just
> > > >
> > > > >
> > > >
> > > > > want to say the current interface does not prevent users from
doing
> > it.
> > > >
> > > > >
> > > >
> > > > > From the perspective of the new sink api, we might not depend on
> > > emitting
> > > >
> > > > >
> > > >
> > > > > records in `NotifyCheckpointComplete`, like using
> > `OperatorCoordinator`
> > > >
> > > > >
> > > >
> > > > > instead.
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > *2. What does the FLIP-147 guarantee?*I think initially this FLIP
> > want
> > > to
> > > >
> > > > >
> > > >
> > > > > achieve two targets:
> > > >
> > > > >
> > > >
>
> > > > > 1. Tasks/Operators exit correctly (as you mentioned the lifecycle
of
> > a
> > > >
> > > > >
> > > >
> > > > > Task/StreamTask/StreamOperator.).
> > > >
> > > > >
> > > >
> > > > > 2. Continue to trigger checkpoint after some tasks for mixed
jobs.
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
>
> > > > > I think the first thing is related to the discussion in
FLINK-21133.
> > > If I
> > > >
> > > > >
> > > >
> > > > > understand correctly, in addition to supporting the tasks /
> > operators
> > > to
> > > >
> > > > >
> > > >
> > > > > exit correctly, now we also want to unify the process of the
tasks
> > and
> > > >
> > > > >
> > > >
> > > > > operators for savepoint / finish.
> > > >
> > > > >
> > > >
> > > > > I think the second thing is orthogonal to the FLINK-21133 because
> > there
> > > >
> > > > are
> > > >
> > > > >
> > > >
> > > > > topologies that have both the bounded and unbounded input.
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > *3. How to unify the operator exit process of FLIP-147 with
> > > >
> > > > >
> > > >
> > > > > stop-with-savepoint?*
> > > >
> > > > >
> > > >
> > > > > I am not very sure about how to do it yet. But if I understand
the
> > > >
> > > > >
> > > >
> > > > > discussion in the jira correctly it needs to introduce some logic
> > into
> > > >
> > > > >
> > > >
>
> > > > > `CheckpointCoordinator`, which responses for triggering “the
unified
> > > >
> > > > >
> > > >
> > > > > operator exit process”. Am I correct?
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > Best,
> > > >
> > > > >
> > > >
> > > > > Guowei
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann wrote:
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > > Thanks for the explanation Yun and Guowei. I have to admit that
I
> > do
> > > >
> > > > not
> > > >
> > > > >
> > > >
>
> > > > > > fully understand why this is strictly required but I think that
we
> > > are
> > > >
> > > > >
> > > >
> > > > > > touching two very important aspects which might have far
fetching
> > > >
> > > > >
> > > >
> > > > > > consequences for how Flink works:
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > 1) Do we want to allow that multiple checkpoints are required
to
> > > >
> > > > >
> > > >
> > > > > > materialize results?
> > > >
> > > > >
> > > >
> > > > > > 2) Do we want to allow to emit records in
> > notifyCheckpointComplete?
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > For 1) I am not sure whether this has been discussed within the
> > > >
> > > > community
> > > >
> > > > >
> > > >
> > > > > > sufficiently. Requiring multiple checkpoints to materialize a
> > result
> > > >
> > > > >
> > > >
> > > > > > because of multi level committers has the consequence that we
> > > increase
> > > >
> > > > > the
> > > >
> > > > >
> > > >
>
> > > > > > latency from checkpoint interval to #levels * checkpoint
interval.
> > > >
> > > > >
> > > >
> > > > > > Moreover, having to drain the pipeline in multiple steps, would
> > break
> > > >
> > > > the
> > > >
> > > > >
> > > >
> > > > > > stop-with-savepoint --drain because which savepoint do you
report
> > to
> > > >
> > > > the
> > > >
> > > > >
> > > >
> > > > > > user?
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > For 2) allowing to send records after the final
> > > >
> > > > notifyCheckpointComplete
> > > >
> > > > >
> > > >
> > > > > > will effectively mean that we need to shut down a topology in
> > > multiple
> > > >
> > > > >
> > > >
> > > > > > steps (in the worst case one operator per checkpoint). This
would
> > be
> > > a
> > > >
> > > > >
> > > >
> > > > > > strong argument for not allowing this to me. The fact that
users
> > can
> > > >
> > > > send
> > > >
> > > > >
> > > >
> > > > > > records after the notifyCheckpointComplete is more by accident
> > than
> > > by
> > > >
> > > > >
> > > >
>
> > > > > > design. I think we should make this a very deliberate decision
and
> > in
> > > >
> > > > > doubt
> > > >
> > > > >
> > > >
>
> > > > > > I would be in favour of a more restrictive model unless there
is a
> > > very
> > > >
> > > > >
> > > >
> > > > > > good reason why this should be supported.
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > Taking also the discussion in FLINK-21133 [1] into account, it
> > seems
> > > to
> > > >
> > > > > me
> > > >
> > > > >
> > > >
> > > > > > that we haven't really understood what kind of guarantees we
want
> > to
> > > >
> > > > give
> > > >
> > > > >
> > > >
> > > > > > to our users and how the final checkpoint should exactly work.
I
> > > >
> > > > > understand
> > > >
> > > > >
> > > >
> > > > > > that this is not included in the first scope of FLIP-147 but I
> > think
> > > >
> > > > this
> > > >
> > > > >
> > > >
> > > > > > is so important that we should figure this out asap. Also
because
> > the
> > > >
> > > > > exact
> > > >
> > > > >
> > > >
> > > > > > shut down behaviour will have to be aligned with the lifecycle
of
> > a
> > > >
> > > > >
> > > >
> > > > > > Task/StreamTask/StreamOperator. And last but not least because
> > other
> > > >
> > > > >
> > > >
> > > > > > features such as the new sink API start building upon a shut
down
> > > model
> > > >
> > > > >
> > > >
> > > > > > which has not been fully understood/agreed upon.
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > [1] https://issues.apache.org/jira/browse/FLINK-21133
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > Cheers,
> > > >
> > > > >
> > > >
> > > > > > Till
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma wrote:
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > > Thanks Yun for the detailed explanation.
> > > >
> > > > >
> > > >
>
> > > > > > > A simple supplementary explanation about the sink case: Maybe
we
> > > >
> > > > could
> > > >
> > > > >
> > > >
> > > > > > use
> > > >
> > > > >
> > > >
> > > > > > > `OperatorCoordinator` to avoid sending the element to the
> > > downstream
> > > >
> > > > >
> > > >
> > > > > > > operator.
> > > >
> > > > >
> > > >
> > > > > > > But I agree we could not limit the users not to emit records
in
> > the
> > > >
> > > > >
> > > >
> > > > > > > `notiyCheckpointComplete`.
> > > >
> > > > >
> > > >
> > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > Best,
> > > >
> > > > >
> > > >
> > > > > > > Guowei
> > > >
> > > > >
> > > >
> > > > > > >
> > > >
> > > > >
> > > >
> > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao
> > > >
> > > > >
> > > >
> > > > > > > wrote:
> > > >
> > > > >
> > > >
> > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > Hi all,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > I'd like to first detail the issue with emitting records in
> > > >
> > > > >
> > > >
> > > > > > > > notifyCheckpointComplete for context. For specific usage,
> > > >
> > > > >
> > > >
> > > > > > > > an example would be for sink, it might want to write some
> > > metadata
> > > >
> > > > >
> > > >
> > > > > > after
> > > >
> > > > >
> > > >
> > > > > > > > all the transactions are committed
> > > >
> > > > >
> > > >
> > > > > > > > (like write a marker file _SUCCESS to the output
directory).
> > This
> > > >
> > > > > case
> > > >
> > > > >
> > > >
> > > > > > is
> > > >
> > > > >
> > > >
> > > > > > > > currently supported via the two level
> > > >
> > > > >
> > > >
> > > > > > > > committers of the new sink API: when received endOfInput(),
> > the
> > > >
> > > > >
> > > >
> > > > > > Committer
> > > >
> > > > >
> > > >
> > > > > > > > wait for another checkpoint to
> > > >
> > > > >
> > > >
> > > > > > > > commits all the pending transactions and emit the list of
> > files
> > > to
> > > >
> > > > > the
> > > >
> > > > >
> > > >
> > > > > > > > GlobalCommitter. The GlobalCommitter
> > > >
> > > > >
> > > >
> > > > > > > > would wait for another checkpoint to also write the
metadata
> > with
> > > >
> > > > 2pc
> > > >
> > > > >
> > > >
> > > > > > > > (Although sometimes 2pc is not needed
> > > >
> > > > >
> > > >
> > > > > > > > for writing metadata, it should be only an optimization and
> > still
> > > >
> > > > >
> > > >
> > > > > > > requires
> > > >
> > > > >
> > > >
> > > > > > > > the Committer do commit before
> > > >
> > > > >
> > > >
> > > > > > > > notifying the global Committer. Also another note is
> > > >
> > > > GlobalCommitter
> > > >
> > > > > is
> > > >
> > > > >
> > > >
> > > > > > > > also added for some other cases
> > > >
> > > > >
> > > >
> > > > > > > > like some sinks want an commiter with dop = 1, like
> > IceBergSink).
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > However, a more general issue to me is that currently we do
> > not
> > > >
> > > > limit
> > > >
> > > > >
> > > >
> > > > > > > > users to not emit records in
> > > >
> > > > >
> > > >
>
> > > > > > > > notifyCheckpointComplete in the API level. The sink case
could
> > be
> > > >
> > > > >
> > > >
> > > > > > viewed
> > > >
> > > > >
> > > >
> > > > > > > > as a special case, but in addition
> > > >
> > > > >
> > > >
> > > > > > > > to this one, logically users could also implement their own
> > cases
> > > >
> > > > > that
> > > >
> > > > >
> > > >
> > > > > > > > emits records in notifyCheckpointComplete.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > Best,
> > > >
> > > > >
> > > >
> > > > > > > > Yun
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > ------------------Original Mail ------------------
> > > >
> > > > >
> > > >
> > > > > > > > Sender:Arvid Heise
> > > >
> > > > >
> > > >
> > > > > > > > Send Date:Fri Feb 12 20:46:04 2021
> > > >
> > > > >
> > > >
> > > > > > > > Recipients:dev
> > > >
> > > > >
> > > >
> > > > > > > > CC:Yun Gao
> > > >
> > > > >
> > > >
> > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After
> > Tasks
> > > >
> > > > >
> > > >
> > > > > > Finished
> > > >
> > > > >
> > > >
> > > > > > > > Hi Piotr,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > Thank you for raising your concern. Unfortunately, I do not
> > have
> > > a
> > > >
> > > > >
> > > >
> > > > > > better
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > idea than doing closing of operators intermittently with
> > > >
> > > > checkpoints
> > > >
> > > > > (=
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > multiple last checkpoints).
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > However, two ideas on how to improve the overall user
> > experience:
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > 1. If an operator is not relying on
notifyCheckpointComplete,
> > we
> > > >
> > > > can
> > > >
> > > > >
> > > >
> > > > > > > close
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > it faster (without waiting for a checkpoint). In general,
I'd
> > > >
> > > > assume
> > > >
> > > > >
> > > >
> > > > > > that
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > almost all non-sinks behave that way.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > 2. We may increase the checkpointing frequency for the last
> > > >
> > > > >
> > > >
> > > > > > checkpoints.
> > > >
> > > > >
> > > >
> > > > > > > We
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > need to avoid overloading checkpoint storages and task
> > managers,
> > > >
> > > > but
> > > >
> > > > > I
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > assume the more operators are closed, the lower the
> > checkpointing
> > > >
> > > > >
> > > >
> > > > > > > interval
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > can be.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > For 1, I'd propose to add (name TBD):
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > default boolean StreamOperator#requiresFinalCheckpoint() {
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > return true;
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > }
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > This means all operators are conservatively (=slowly)
closed.
> > For
> > > >
> > > > > most
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > operators, we can then define their behavior by overriding
in
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > AbstractUdfStreamOperator
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > @Override
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint()
{
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > return userFunction instanceof CheckpointListener;
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > }
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > This idea can be further refined in also adding
> > > >
> > > > > requiresFinalCheckpoint
> > > >
> > > > >
> > > >
> > > > > > > to
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > CheckpointListener to exclude all operators with UDFs that
> > > >
> > > > implement
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > CheckpointListener but do not need it for 2pc.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > @Override
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint()
{
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > return userFunction instanceof CheckpointListener &&
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > ((CheckpointListener)
userFunction).requiresFinalCheckpoint();
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > }
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > That approach would also work for statebackends/snapshot
> > > strategies
> > > >
> > > > >
> > > >
> > > > > > that
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > require some 2pc.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > If we can contain it to the @PublicEvolving StreamOperator,
it
> > > >
> > > > would
> > > >
> > > > > be
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > better of course.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > Best,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > Arvid
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > wrote:
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > Hey,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > I would like to raise a concern about implementation of
the
> > > final
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > checkpoints taking into account operators/functions that
are
> > > >
> > > > >
> > > >
> > > > > > > implementing
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > two phase commit (2pc) protocol for exactly-once
processing
> > > with
> > > >
> > > > > some
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > external state (kept outside of the Flink). Primarily
> > > >
> > > > exactly-once
> > > >
> > > > >
> > > >
> > > > > > > sinks.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > First of all, as I understand it, this is not planned in
the
> > > >
> > > > first
> > > >
> > > > >
> > > >
> > > > > > > > version
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > of this FLIP. I'm fine with that, however I would
strongly
> > > >
> > > > > emphasize
> > > >
> > > > >
> > > >
> > > > > > > this
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > in every place we will be mentioning FLIP-147 efforts.
This
> > is
> > > >
> > > > >
> > > >
> > > > > > because
> > > >
> > > > >
> > > >
> > > > > > > > me,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > as a user, upon hearing "Flink supports checkpointing
with
> > > >
> > > > bounded
> > > >
> > > > >
> > > >
> > > > > > > > inputs"
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > I would expect 2pc to work properly and to commit the
> > external
> > > >
> > > > side
> > > >
> > > > >
> > > >
> > > > > > > > effects
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > upon finishing. As it is now, I (as a user) would be
> > surprised
> > > >
> > > > > with a
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > silent data loss (of not committed trailing data). This
is
> > > just a
> > > >
> > > > >
> > > >
> > > > > > > remark,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > that we need to attach this warning to every blog
> > > >
> > > > >
> > > >
> > > > > > > post/documentation/user
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > mailing list response related to "Support Checkpoints
After
> > > Tasks
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > Finished". Also I would suggest to prioritize the follow
up
> > of
> > > >
> > > > >
> > > >
> > > > > > > supporting
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 2pc.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > Secondly, I think we are missing how difficult and
> > problematic
> > > >
> > > > will
> > > >
> > > > >
> > > >
> > > > > > be
> > > >
> > > > >
> > > >
> > > > > > > > 2pc
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > support with the final checkpoint.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > For starters, keep in mind that currently 2pc can be
> > > implemented
> > > >
> > > > by
> > > >
> > > > >
> > > >
> > > > > > > users
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > using both `@Public` APIs as functions and
`@PublicEvolving`
> > > >
> > > > >
> > > >
> > > > > > operators
> > > >
> > > > >
> > > >
> > > > > > > in
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > any place in the job graph. It's not limited to only the
> > sinks.
> > > >
> > > > For
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > example users could easily implement the `AsynFunction`
(for
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > `AsyncWaitOperator`) that is using 2pc based on the
> > > >
> > > > >
> > > >
> > > > > > > `CheckpointListener`
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > interface. I'm not saying it's common, probably just a
tiny
> > > >
> > > > > minority
> > > >
> > > > >
> > > >
> > > > > > of
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > users are doing that (if any at all), but nevertheless
> > that's
> > > >
> > > > >
> > > >
> > > > > > possible
> > > >
> > > > >
> > > >
> > > > > > > > and
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > currently (implicitly?) supported in Flink.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > Next complication is the support of bounded streams
> > > >
> > > > >
> > > >
> > > > > > (`BoundedOneInput`
> > > >
> > > > >
> > > >
> > > > > > > or
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > `BoundedMultiInput` interfaces) and the closing/shutdown
> > > >
> > > > procedure
> > > >
> > > > > of
> > > >
> > > > >
> > > >
> > > > > > > the
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > operators. Currently it works as follows:
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > 1. `endOfInput` is called on the first operator in the
chain
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 2. We quiesce the processing timers
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > >
(`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> > > for
> > > >
> > > > > the
> > > >
> > > > >
> > > >
> > > > > > > > first
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > operator, so no new timers will be triggered
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 3. We wait for the already fired timers to finish
executing
> > > >
> > > > > (spinning
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > mailbox loop)
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 4. We are closing the first operator
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 5. We go to the next (second) operator in the chain and
> > repeat
> > > >
> > > > the
> > > >
> > > > >
> > > >
> > > > > > > steps
> > > >
> > > > >
> > > >
> > > > > > > > 1.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > to 5.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > This is because operators can emit data after processing
> > > >
> > > > >
> > > >
> > > > > > `endOfInput`,
> > > >
> > > > >
> > > >
> > > > > > > > from
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > timers, async mailbox actions and inside the `close`
method
> > > >
> > > > itself.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > Now the problem is to support the final checkpoint with
2pc,
> > we
> > > >
> > > > > need
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > trigger `snapshotState` and `notifyCheckpointComplete`
call
> > at
> > > >
> > > > the
> > > >
> > > > >
> > > >
> > > > > > very
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > least only after `endOfInput` call on the operator.
Probably
> > > the
> > > >
> > > > > best
> > > >
> > > > >
> > > >
> > > > > > > > place
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > would be in between steps 3. and 4. However that means,
we
> > > would
> > > >
> > > > be
> > > >
> > > > >
> > > >
> > > > > > > > forced
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > to wait for steps 1. to 3. to finish, then wait for a
next
> > > >
> > > > > checkpoint
> > > >
> > > > >
> > > >
> > > > > > > to
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > trigger AND complete, before finally closing the head
> > operator,
> > > >
> > > > and
> > > >
> > > > >
> > > >
> > > > > > > only
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > then we can start closing the next operator in the chain:
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > 1. `endOfInput` is called on the first operator in the
chain
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 2. We quiesce the processing timers
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > >
(`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> > > for
> > > >
> > > > > the
> > > >
> > > > >
> > > >
> > > > > > > > first
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > operator, so no new timers will be triggered
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 3. We wait for the already fired timers to finish
executing
> > > >
> > > > > (spinning
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > mailbox loop)
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > *3b. We wait for one more checkpoint to trigger and for
the
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > `notifyCheckpointComplete` RPC.*
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 4. We are closing the first operator
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 5. We go to the next (second) operator in the chain and
> > repeat
> > > >
> > > > the
> > > >
> > > > >
> > > >
> > > > > > > steps
> > > >
> > > > >
> > > >
> > > > > > > > 1.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > to 5.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > That means, we can close one operator per successful
> > > checkpoint.
> > > >
> > > > To
> > > >
> > > > >
> > > >
> > > > > > > close
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 10 operators, we would need 10 successful checkpoints.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > I was thinking about different approaches to this
problem,
> > and
> > > I
> > > >
> > > > >
> > > >
> > > > > > > couldn't
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > find any viable ones. All I could think of would break
the
> > > >
> > > > current
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > `@Public` API and/or would be ugly/confusing for the
users.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > For example a relatively simple solution, to introduce a
> > > >
> > > > `preClose`
> > > >
> > > > >
> > > >
> > > > > > or
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > `flush` method to the operators, with a contract that
after
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > `flush`, operators would be forbidden from emitting more
> > > records,
> > > >
> > > > > so
> > > >
> > > > >
> > > >
> > > > > > > that
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > we can replace step 4. with this `flush` call, and then
> > having
> > > a
> > > >
> > > > >
> > > >
> > > > > > single
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > checkpoint to finish 2pc for all of the operators inside
the
> > > >
> > > > chain,
> > > >
> > > > >
> > > >
> > > > > > > > doesn't
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > work. Sheer fact of adding this `flush` method and
changing
> > the
> > > >
> > > > >
> > > >
> > > > > > > contract
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > would break the current API and Yun Gao has pointed out
to
> > me,
> > > >
> > > > that
> > > >
> > > > >
> > > >
> > > > > > we
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > either already support, or want to support operators that
> > are
> > > >
> > > > >
> > > >
> > > > > > emitting
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > records from within the `notifyCheckpointComplete` call:
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > Yun Gao:
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > like with the new sink api there might be writer ->
> > committer
> > > >
> > > > ->
> > > >
> > > > >
> > > >
> > > > > > > global
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > committer, the committer would need to wait for the last
> > > >
> > > > checkpoint
> > > >
> > > > >
> > > >
> > > > > > to
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > commit
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > the last piece of data, and after that it also need to
> > emit
> > > the
> > > >
> > > > >
> > > >
> > > > > > list
> > > >
> > > > >
> > > >
> > > > > > > of
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > transactions get committed to global committer to do some
> > > >
> > > > >
> > > >
> > > > > > finalization
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > logic.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > So it wouldn't solve the problem (at least not fully).
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > I don't know if anyone has any better ideas how to solve
> > this
> > > >
> > > > >
> > > >
> > > > > > problem?
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > Piotrek
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > pt., 15 sty 2021 o 14:57 Yun Gao
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > napisał(a):
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > Hi Aljoscha,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > I think so since we seems to do not have other
divergence
> > and
> > > >
> > > > new
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > objections now. I'll open the vote then. Very thanks!
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > Best,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > Yun
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
------------------------------------------------------------------
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > From:Aljoscha Krettek
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > Send Time:2021 Jan. 15 (Fri.) 21:24
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > To:dev
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints
After
> > > Tasks
> > > >
> > > > >
> > > >
> > > > > > > > Finished
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > Thanks for the summary! I think we can now move towards
a
> > > >
> > > > [VOTE]
> > > >
> > > > >
> > > >
> > > > > > > > thread,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > right?
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > On 2021/01/15 13:43, Yun Gao wrote:
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > >1) For the problem that the "new" root task
coincidently
> > > >
> > > > > finished
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > > >before getting triggered successfully, we have listed
two
> > > >
> > > > > options
> > > >
> > > > >
> > > >
> > > > > > in
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > >the FLIP-147[1], for the first version, now we are not
> > tend
> > > to
> > > >
> > > > > go
> > > >
> > > > >
> > > >
> > > > > > > with
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > >the first option that JM would re-compute and
re-trigger
> > new
> > > >
> > > > >
> > > >
> > > > > > sources
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > >when it realized some tasks are not triggered
> > successfully.
> > > >
> > > > This
> > > >
> > > > >
> > > >
> > > > > > > > option
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > >would avoid the complexity of adding new PRC and
> > duplicating
> > > >
> > > > > task
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > >states, and in average case it would not cause too
much
> > > >
> > > > > overhead.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > > You wrote "we are *not* tend to go with the first
option",
> > > but
> > > >
> > > > I
> > > >
> > > > >
> > > >
> > > > > > > think
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > you meant wo write "we tend to *now* go with the first
> > > option",
> > > >
> > > > >
> > > >
> > > > > > > right?
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > That's also how it is in the FLIP, I just wanted to
> > clarify
> > > for
> > > >
> > > > > the
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > mailing list.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > >
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > >
> > >
> >
> >
>
>
>

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

Posted by Piotr Nowojski <pn...@apache.org>.
Hi,

Thanks for reminding me. I think FLIP-147 will have to deal in one way or
another with the (re?)emitting MAX_WATERMARK. There is a pre-existing issue
that watermarks are not checkpointed/stored on state, and there was/is now
clear answer how we should handle this as far as I remember. One
problematic case are two/multiple input tasks or UnionInputGate, where
combined watermark is the min of all inputs (held in memory). The problem
so far is a bit benign, as after recovery we are losing the combined
watermark value, but it's being slowly/lazily restored, as new watermarks
are sent from the sources. With finished sources that won't be a case.

I've spent more time thinking about the two phase commit issue, and I agree
it would be best to have a single final checkpoint for the operators. The
more I think about it, the more I'm leaning toward the
"OperatorCoordinator" approach. Yes, it would need the extra complexity of
running user code on the JobManager, but that's already at least partially
done (FLIP-27). On the other hand it would allow us to keep the runtime
code cleaner and simpler I think. It's still an invasive change.

1. We forbid emitting records from
`notifyCheckpointComplete`/`snapshotState` calls, or at the very least from
the final calls.
2. We need to clarify/clean up contracts for both operators AND user
functions when it comes to `endOfInput()`, `close()` and `dispose()`
methods. For example:
  a) we keep `endOfInput()` on the operators level and add an equivalent of
it to the user functions
  b) we move the "flushing buffered records" part of the `close()` contract
to `endOfInput()`
  c) we remove `close()` methods altogether (IMO it's better to have an
explicit false conflict, rather than a silent real one)
  d) we keep `dispose()` on the operator level for release resources, and
we add such method to the user functions
3. We change (or add another version) a return type of either `void
snapshotState(...)` or `void notifyCheckpointComplete(...)` calls, so that
they will return some kind of `GlobalCommitHandle`.

Especially this point 2. is intrusive. Now, putting all of that together.
When a task receives a logical "end of input" signal (current
`EndOfPartitionEvent`), it begins shutting down procedure (let's assume we
create `GlobalCommitHandle` in `snapshotState`).

0. n = 1
1. endOfInput on the n'th operator
2. quisec timers and mailbox executor for n'th operator
// after this point n'th operator is forbidden to emit any new records
3. n += 1, and go to 1., until we handle all of the operators
4. wait for final checkpoint, issue `snapshotState` and collect
`GlobalCommitHandle`s and send them to the JM
5. JM/CheckpointCoordinator collects all `GlobalCommitHandles`. They are an
integral part of the checkpoint. They could be added to the respective
`OperatorCoordinator`s via `handleEventFromOperator` as an `OperatorEvent`.
6. When checkpoint is considered completed, `notifyCheckpointComplete` are
being issued, both on the operators, and `OperatorCoordinator` - local
commit and global commits would be happening at the same time, or even
global commit code would be executed before local final commit in this
version.
7. Only now can we finally dispose of all operators in the task.

If there is a failure, especially before we manage to perform all
`notifyCheckpointComplete` calls (for example on the
`OperatorCoordinator`), we would need to recover from last "completed"
checkpoint, recover attached `GlobalCommitHandle`s , and re-commit them. It
also means we need to recover already finished operators, that will never
process any records, just to issue the final `notifyCheckpointComplete`
call and make sure they commit their external side effects.

There are at least a couple of questions about this general idea:
I. Naming of the methods (I suggested to drop `close()` and keep in the
operators and introduce in the functions `endOfInput()` and `dispose()`
II. When should the `GlobalCommitHandle` be created? Should it be returned
from `snapshotState()`, `notifyCheckpointComplete()` or somewhere else?
III. What should be the ordering guarantee between global commit and local
commit, if any? Actually the easiest to implement would be undefined, but
de facto global commit happening before local commits (first invoke
`notifyCheckpointComplete()` on the `OperatorCoordinator` and either after
or in parallel send `notifyCheckpointComplete()` RPCs). As far as I can
tell, undefined order should work for the use cases that I'm aware of.
IV. Can we reuse `OperatorCoordinator` abstraction for this purpose?
Currently it's being used to induce checkpoints with FLIP-27 sources as far
as I understand, here we would be using it for different purposes.

Best,
Piotrek

pon., 1 mar 2021 o 07:30 Yun Gao <yu...@aliyun.com> napisał(a):

> (Sorry that I repeat this mail since the last one is not added into the
> same mail list thread,
> very sorry for the inconvenience)
>
>  Hi all,
>
>     Very thanks for all the deep thoughts!
>
> > How to implement the stop-with-savepoint --drain/terminate command with
> > this model: One idea could be to tell the sources that they should stop
> > reading. This should trigger the EndOfPartitionEvent to be sent
> > downstream.
> > This will transition all operators into the TERMINATING state.
> >
> > Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
> > returned. To achieve above, possible works should be required:
> > * Promote  `EndOfPartitionEvent` from `Task` to `StreamTask`. This may
> > have some interferences with BatchTask or network io stack.
> > * Or introducing stream task level `EndOfUserRecordsEvent`(from PR#14831
> > @Yun @Piotr)
> > * Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.
>
> I also have similar concern with Kezhu for the issue whether do we need to
> introduce a new message
> to notify the operators to endOfInput/close ? The main concerns of reusing
> that EndOfPartitionEvent
> is that
> 1. The EndOfPartitionEvent is currently emitted in Task instead of
> StreamTask, we would need some
> refactors here.
> 2. Currently the InputGate/InputChannel would be released after the
> downstream tasks have received
> EndOfPartitionEvent from all the input channels, this would makes the
> following checkpoint unable to
> perform since we could not emit barriers to downstream tasks ?
>
> Regarding the MAX_WATERMARK, I still not fully understand the issue since
> it seems to me
> that now Flink won't snapshot the watermark now? If the job failover, the
> window operator
> would reload all the pending windows before flushed by MAX_WATERMARK and
> when the
> job finish again, it would re-emit the MAX_WATERMARK?
>
> Best,
> Yun
>
>
> ------------------------------------------------------------------
> From:Kezhu Wang <ke...@gmail.com>
> Send Time:2021 Mar. 1 (Mon.) 01:26
> To:Till Rohrmann <tr...@apache.org>
> Cc:Piotr Nowojski <pi...@gmail.com>; Guowei Ma <
> guowei.mgw@gmail.com>; dev <de...@flink.apache.org>; Yun Gao <
> yungao.gy@aliyun.com>; jingsonglee0@gmail.com <ji...@gmail.com>
> Subject:Re: Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> Finished
>
> In “stop-with-savepoint —drain”, MAX_WATERMARK is not an issue. For normal
> finishing task, not allowing unaligned checkpoint does not solve the
> problem as MAX_WATERMARK could be persisted in downstream task. When
> scenario @Piotr depicted occurs, downstream(or further downstream) window
> operator will count all inputs as late.
>
> > If we ensure that the MAX_WATERMARK is only persisted in state if a
> recovery will trigger immediately the shut down of this operator, then it
> shouldn't be an issue.
>
> You are right in case the assumption holds, I have same thought as you
> before. But I am kind of worry about whether it is too prefect to be
> fragile. This requires strong guarantee from implementation that a recovery
> from TERMINATING stage should go directly to that stage.
>
> > I think the MAX_WATERMARK event should be sent either just before or with
> the EndOfPartitionEvent.
>
> I prefer “with the EndOfPartitionEvent”. EndOfPartitionEvent itself already
> carry what ending MAX_WATERMARK try to express. May be we can reuse it ?
> @Piotr
>
> A preview work[1] from @Yun in PR#14831 explains EndOfPartitionEvent as
> checkpoint barrier if there are pending checkpoints.
>
>
> [1]:
>
> https://github.com/gaoyunhaii/flink/commit/1aaa80fb0afad13a314b06783c61c01b9414f91b#diff-d4568b34060bc589cd1354bd125c35aca993dd0c9fe9e4460dfed73501116459R177
>
>
> Best,
> Kezhu Wang
>
> On February 28, 2021 at 21:23:31, Till Rohrmann (trohrmann@apache.org)
> wrote:
>
> I think you are right with the problem of endOfInput. endOfInput should not
> be used to commit final results. In fact if this termination fails then we
> might end up in a different outcome of the job which is equally valid as
> the one before the failure.
>
> Concerning unaligned checkpoints, I think they don't play well together
> with draining a streaming pipeline. The problem is that in the draining
> case you want to process all records which are still in flight but
> unaligned checkpoints don't guarantee this as they can jump in flight
> records.
>
> I think the MAX_WATERMARK event should be sent either just before or with
> the EndOfPartitionEvent. If we ensure that the MAX_WATERMARK is only
> persisted in state if a recovery will trigger immediately the shut down of
> this operator, then it shouldn't be an issue.
>
> Cheers,
> Till
>
> On Sun, Feb 28, 2021 at 9:06 AM Kezhu Wang <ke...@gmail.com> wrote:
>
> > Hi Till,
> >
> > Just for bookkeeping, some observations from current implementation.
> >
> > > With this model, the final checkpoint is quite simple because it is
> > ingrained in the lifecycle of an operator. Differently said an operator
> > will only terminate after it has committed its side effects and seen the
> > notifyCheckpointComplete message (if it is stateful).
> >
>
> > Currently, we could not mark this operator(or subtask) as terminated since
> > result of `notifyCheckpointComplete`(possible side effect committing) is
> > not taken into account of the belonging checkpoint. The job has to run to
> > next safe point(finished or next checkpoint success) to be marked as
> > “terminated”.
> >
> > > How to implement the stop-with-savepoint --drain/terminate command with
> > this model: One idea could be to tell the sources that they should stop
> > reading. This should trigger the EndOfPartitionEvent to be sent
> > downstream.
> > This will transition all operators into the TERMINATING state.
> >
> > Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
> > returned. To achieve above, possible works should be required:
> > * Promote  `EndOfPartitionEvent` from `Task` to `StreamTask`. This may
> > have some interferences with BatchTask or network io stack.
> > * Or introducing stream task level `EndOfUserRecordsEvent`(from PR#14831
> > @Yun @Piotr)
> > * Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.
> >
> > Besides this, I would like to quote some discussion from FLINK-21467
> > between @Piotr and me:
> >
> > From @Piotr
> > > Note, that it's not only that endOfInput can be called multiple times.
>
> > There is a very remote possibility that the following scenario will happen:
> > 1. checkpoint is taken (successfully)
> > 2. sources are finishing
> > 3. endOfInput is issued
> > 4. job fails
> > 5. job restarts to checkpoint 1.
> > 6. after failover, because of some non deterministic logic in the source,
> > sources are not finishing
> >
> > From me
> > > But I think there is little work Flink can do to cope with this kind of
> > issues. The checkpoint could be a savepoint triggered from user side and
> > the "non deterministic logic" could be a change from user(eg. changing of
> > stoppingOffsets in KafkaSource).
> >
>
> > > I think the "non deterministic logic" could cause trouble in combination
> > with unaligned checkpoint and downstream window operator. Unaligned
> > checkpoint will persist "MAX_WATERMARK" in state, after restarting,
> > "MAX_WATERMARK" will cause downstream window operator ignores all future
> > inputs.
> >
> > FLIP-147 demands no new records from end-of-stream-flushing, but source
> > will emit “MAX_WATERMARK” on ending. Previously, I thought it is not a
>
> > valid issue, but turn out that it could cause trouble under scenario listed
> > by @Piotr if I am not wrong.
> >
> >
> > PR#14831: https://github.com/apache/flink/pull/14831
> > FLINK-21467: https://issues.apache.org/jira/browse/FLINK-21467
> >
> >
> > Best,
> > Kezhu Wang
> >
> > On February 27, 2021 at 18:12:20, Till Rohrmann (trohrmann@apache.org)
> > wrote:
> >
> > Thanks for all your thoughts. I think we should further think through
>
> > whether to allow checkpoints after an operator has emitted all its records
> > (e.g. after close is called currently) or not. I think by doing this we
> > would nicely decouple the checkpoint taking from the operator lifecycle
> > and
> > wouldn't need special checkpoints/savepoints for the final checkpoint and
> > stop-with-savepoint --drain. Let me try to explain this a bit more
> > detailed.
> >
> > If we say an operator has the rough lifecycle RUNNING => TERMINATING =>
> > TERMINATED where we go from RUNNING into TERMINATING after we have seen
> > the
> > EndOfPartitionEvent and flushed all our records. The operator goes from
> > TERMINATING => TERMINATED if it has persisted all its possible side
> > effects. Throughout all states, it is possible to trigger a checkpoint. A
> > stateless operator will immediately go from TERMINATING to TERMINATED
> > whereas a stateful operator would wait for another checkpoint to be
> > triggered and successfully completed (notifyCheckpointComplete).
> >
> > With this model, the final checkpoint is quite simple because it is
> > ingrained in the lifecycle of an operator. Differently said an operator
> > will only terminate after it has committed its side effects and seen the
>
> > notifyCheckpointComplete message (if it is stateful). Here it is important
> > to note that in the streaming case, different bounded operators can
> > terminate at different times. They don't have to terminate all with the
> > same checkpoint.
> >
> > How to implement the stop-with-savepoint --drain/terminate command with
> > this model: One idea could be to tell the sources that they should stop
> > reading. This should trigger the EndOfPartitionEvent to be sent
> > downstream.
>
> > This will transition all operators into the TERMINATING state. Next the JM
> > can trigger a checkpoint to shut the operators down and not to wait for
> > the
> > next automatic checkpoint trigger event.
> >
> > By allowing checkpoints throughout the entire lifecycle of an operator we
> > disallow sending records from notifyCheckpointComplete because this
> > message
> > will also be sent in the state TERMINATING where an operator has already
> > produced all of its records.
> >
> > What do you think? Maybe this model overlooks some important cases. One
> > downside is that we will break the operator API with changing the
> > lifecycle
> > of an operator.
> >
> > Cheers,
> > Till
> >
> >
> >
> > On Fri, Feb 26, 2021 at 7:29 AM Kezhu Wang <ke...@gmail.com> wrote:
> >
> > > Hi Yun,
> > >
> > > The termination phase I drafted depends on a de-fact that
>
> > > `notifyCheckpointComplete` is ignored currently after `close`. Thus, any
> > > external materialization inside `close`(including `endInput`) or
> > > `notifyCheckpointComplete` is either not guaranteed or not committed.
> > >
> > > I tried to emphasize no-breaking changes and post-pone migration in
> > later
> > > releases for this termination phase. But now, I am kind of worry about
> > > whether it will cause long-term maintenance hell.
> > >
> > > Personally, I think allowing checkpoint after `close`(@Till proposed
> > this
> > > in FLINK-21133) could be the minimal change. But there are concerns:
> > > * It will break some existing code possibly in a silent way.(@Piotr
> > pointed
> > > this already in FLINK-21133)
> > > * I think the name `close` is kind of misleading. (@Piotr suggested
> > > renaming this to `finish` to not break code silently in FLINK-21133)
> > >
> > > > stop-with-savepoint --drain would wait for a specific savepoint
> > >
> > > For stop-with-savepoint, the checkpoint barrier is already created or
> > > received there.
> > >
> > >
> > > Best,
> > > Kezhu Wang
> > >
> > > On February 25, 2021 at 21:02:30, Yun Gao (yungao.gy@aliyun.com
> ) wrote:
> > >
> > > Hi all,
> > >
> > > Very thanks for the discussions!
> > >
> > >
> > >
> > > A. Regarding how to avoid emitting records in notifyCheckpointComplete:
> > >
> > > Currently the structure of a new sink is writer -> committer -> global
> > > committer and the paralellism of global committer
> > > must be one. By design it would be used in several cases:
> > > 1. writer -> committer: for normal sinks that write pending data in
> > writer
> > > and commit pending data in committer.
> > > 2. writer -> global committer: for sinks require the committer's
> > > parallelism be 1, like iceberg sink which
> > > relies on optimistic lock to commit data hopes to reduce the conflicts.
> > > 3. writer -> committer -> global committer: for sinks that also need to
> > > write meta data (like _SUCCESS file
> > > or add record in hive metastore) in global committer.
> > >
> > > The third case would cause the cascade commit problem. To overcome this
> > > problem, we would like to
> > > 1. Previously the global committer also support writing metadata with
> > 2pc.
> > > Now we disable this mode
> > > and always rely on the property that writing metadata is repeatable.
> > With
> > > this limitation there should no
> > > need of cascade commit, but the committer would still emit records in
> > > notifyCheckpointComplete.
> > > 2. We further move global committer in the case 3 to the operator
> > > coordinator. Then the committer would
> > > not need to emit records, but communicates with its operator
> > coordinator.
> > >
> > > One core issue for using the OperatorCoordinator is how to keep the
> > > communication between the operator
> > > and the OperatorCoordinator exactly-once. Since the message is always
> > from
> > > the oeprator to the OperatorCoordinator
> > > in this case, we would only need to bookkeep the message sent between
> > the
> > > OperatorCoordinator takes snapshot and
> > > the Operator takes snapshot in the state of the Operator.
> > >
> > > On how to achieve the change in detail we would still need some think,
> > it
> > > currently seems we would have to had
> > > some modification to the current new sink api.
> > >
> > >
> > >
> > > B. Regarding the stop-with-savepoint --drain
> > >
> > > Very thanks @Piotr for the further explanation and now I realize I have
> > > understand wrongly for the semantics of
> > > stop-with-savepoint --drain. Now I think that the problem should be we
> > > should also include the records produced in
> > > `endOfInput()` and `close()` also in the last savepoint, am I correct?
> > If
> > > so, it seems we still have some undetermined options for
> > > the lifecycle of the operator, like in Kezhu's proposal the close()
> > happens
> > > at last, but it seems close() might also emit records (
> > > so now the operator are closed with op1's close() -> op2's endOfInput()
> > ->
> > > op2's close() -> op3's endOfinput -> ...) ?
> > >
> > > And on the other side, as Kezhu has also proposed, perhapse we might
> > have
> > > the stop-with-savepoint --drain and normal exit in the same process,
> > > but have slightly difference in that stop-with-savepoint --drain would
> > wait
> > > for a specific savepoint and in normal exit, the operator
> > > might wait for arbitrary checkpoint. If we could achieve not emitting
> > > records in notifyCheckpointComplete, stop-with-savepoint --drain could
> > > be done with one savepoint, and for the normal exit, the operator would
> > not
> > > need to wait for other slow operators to exit.
> > >
> > > Best,
> > > Yun
> > >
> > >
> > >
> > > ------------------Original Mail ------------------
> > > *Sender:*Kezhu Wang <ke...@gmail.com>
> > > *Send Date:*Thu Feb 25 15:11:53 2021
> > > *Recipients:*Flink Dev <de...@flink.apache.org>, Piotr Nowojski <
> > > piotr.nowojski@gmail.com>
> > > *CC:*Guowei Ma <gu...@gmail.com>, jingsonglee0@gmail.com <
> > > jingsonglee0@gmail.com>
> > > *Subject:*Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > > Finished
> > >
> > > > Hi all, thanks for driving this and especially Piotr for re-active
> > this
> > > >
> > > > thread.
> > > >
> > > >
> > > >
> > > > First, for `notifyCheckpointComplete`, I have strong preference
> > towards
> > > >
> > > > "shut down the dataflow
> > > >
> > > > pipeline with one checkpoint in total", so I tend to option dropping
> > > "send
> > > >
> > > > records" from
> > > >
> > > > `notifyCheckpointComplete` for next level committing in pipeline, if
> > we
> > > >
> > > > ever support it. Without this,
> > > >
> > > > we are unable to stop a pipeline manually with all results
> > materialized.
> > > >
> > > >
> > > >
> > > > Second, for shutdown unification of `stop-with-savepoint --drain` and
> > > >
> > > > FLIP-147, I draft following
> > > >
> > > > phase based on emerging proposals(mainly by Piotr and Till) in this
> > > thread
> > > >
> > > > and FLINK-21133.
> > > >
> > > >
> > > >
> > > > ```java
> > > >
> > > > // StreamTask termination phase
> > > >
> > > > finish()(Call StreamOperator.finish in chaining order)
> > > >
>
> > > > advanceToEndOfEventTime()(nop if for no-source inputs, this could also
> > be
> > > >
> > > > done in input processor or finish ?)
> > > >
> > > > if (there-is-a-pending-terminate-savepoint) {
> > > >
> > > > triggerBarrierForDownStream();
> > > >
> > > > waitCheckpointComplete();
> > > >
> > > > } else if (require-two-phase-commit-in-shutdown) {
> > > >
> > > > waitFinalCheckpoint();
> > > >
> > > > waitCheckpointComplete();
> > > >
> > > > }
> > > >
> > > > // TODO: Ignore/Cancel all checkpoints from this point(or hit to
> > > >
> > > > coordinator in final checkpoint response ?).
> > > >
> > > > close();
> > > >
> > > > ```
> > > >
> > > >
> > > >
> > > > Branches in `if` could reside in different code paths, but the effect
> > > >
> > > > should be similar to above.
> > > >
> > > > The divergence reflects fact that "stop-with-savepoint --drain" is
> > > >
> > > > triggered already while we need
> > > >
> > > > to wait for final checkpoint in natural input exhausting.
> > > >
> > > >
> > > >
> > > > With carefully chosen default functions, we will not break existing
> > > >
> > > > interfaces.
> > > >
> > > > * `StreamOperator.finish`
> > > >
> > > > * `RichFunction.finish`
> > > >
> > > >
> > > >
> > > > For existing codes:
> > > >
> > > > 1. "stop-with-savepoint" with no migration, it should behaves as
> > before.
> > > >
> > > > 2. "require-two-phase-commit-in-shutdown" evaluates to false and no
> > > >
> > > > migration, it is same as before.
> > > >
> > > > 3. "require-two-phase-commit-in-shutdown" evaluates to true and no
> > > >
> > > > migration, records in `close` will
> > > >
> > > > lose.
> > > >
> > > >
> > > >
> > > > For case#3:
> > > >
> > > > A. No external materialization in `StreamOperator.close`: Nothing to
> > > lose.
> > > >
> > > > B. External materialization only in `StreamOperator.close`: I don't
> > think
> > > >
> > > > Flink ever claimed that there is
> > > >
> > > > any guarantee for this situation.
> > > >
> > > > C. External materialization in `notifyCheckpointComplete` from
> > > >
> > > > `StreamOperator.close`: But due to fact that
> > > >
> > > > `notifyCheckpointComplete` was ignored after operator
> > > >
> > > > closed(FLINK-16383), so there will be no external
> > > >
> > > > materialization to lose.
> > > >
> > > >
> > > >
> > > > Then, we could recommend users to migrate possible
> > > “end-of-stream-flushing”
> > > >
> > > > from “close" to “finish”.
> > > >
> > > >
> > > >
> > > > Best,
> > > >
> > > > Kezhu Wang
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > On February 24, 2021 at 23:47:36, Piotr Nowojski (
> > > piotr.nowojski@gmail.com
> > > > )
> > > >
> > > > wrote:
> > > >
> > > >
> > > >
> > > > Thanks for the reponses Guowei and Yun,
> > > >
> > > >
> > > >
> > > > Could you elaborate more/remind me, what does it mean to replace
> > emitting
> > > >
>
> > > > results from the `notifyCheckpointComplete` with `OperatorCoordinator`
> > > >
> > > > approach?
> > > >
> > > >
> > > >
> > > > About the discussion in FLINK-21133 and how it relates to FLIP-147.
> > You
> > > are
> > > >
>
> > > > right Yun gao, that in case of `stop-with-savepoint --drain` the whole
> > > job
> > > >
> > > > finishes, while in FLIP-147 some of the sources can work for
> > arbitrarily
> > > >
>
> > > > long time after others have already finished. However from the runtime
> > > >
> > > > perspective, especially how to close a Task and it's operators, there
> > is
> > > no
> > > >
> > > > difference between those two cases. In both cases, we need to end
> > input,
> > > >
>
> > > > shut down processing time timers and commit external side effects (two
> > > >
> > > > phase commit support) before exiting the task. That's how the
> > discussion
> > > >
> > > > about the behaviour of "stop-with-savepoint" was connected with
> > FLIP-147.
> > > >
> > > >
> > > >
> > > > Currently on master, "stop-with-savepoint --drain" drains/flushes
> > > buffered
> > > >
>
> > > > records and deals correctly with timers, but all of that happens AFTER
> > > >
> > > > savepoint was completed. So any records flushed from the operators
> > during
> > > >
> > > > endOfInput/close/shutting down processing timers are never committed
> > to
> > > >
> > > > external systems. This is exactly the same problem as the "two phase
> > > >
>
> > > > commit" problem of FLIP-147, that should have the same solution and it
> > > >
>
> > > > should be solved at the same time. For example if we go with the *very
> > > >
> > > > undesirable* "one closed operator per one completed
> > > checkpoint/savepoint",
> > > >
> > > > in both cases CheckpointCoordinator, Scheduler and Task would need to
> > > keep
> > > >
> > > > the task alive and keep triggering checkpoints for that task, until
> > all
> > > >
> > > > operators in the operator chain are closed (one closed operator per
> > one
> > > >
> > > > completed checkpoint).
> > > >
> > > >
> > > >
> > > > Piotrek
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > śr., 24 lut 2021 o 11:30 Yun Gao napisał(a):
> > > >
> > > >
> > > >
> > > > > Hi Till, Guowei,
> > > >
> > > > >
> > > >
> > > > > Very thanks for initiating the disucssion and the deep thoughts!
> > > >
> > > > >
> > > >
>
> > > > > For the notifyCheckpointComplete, I also agree we could try to avoid
> > > >
> > > > > emitting new records in notifyCheckpointComplete via using
> > > >
> > > > > OperatorCoordinator
> > > >
> > > > > for new sink API. Besides, the hive sink might also need some
> > > >
> > > > modification
> > > >
> > > > > for it also emits records in notifyCheckpointComplete.
> > > >
> > > > >
> > > >
>
> > > > > For unifying the process of stopping with savepoint and finished due
> > to
> > > >
> > > > > all records
> > > >
> > > > > are processed, I also agree with that unifying would always be
> > better
> > > if
> > > >
> > > > > we could achieve,
> > > >
> > > > > but I'm still not fully catch up with the implementation: Based on
> > the
> > > >
> > > > > discussion in FLINK-21133,
> > > >
>
> > > > > my understanding is that for stopping with savepoint, now we want to
> > > >
> > > > first
> > > >
> > > > > stop the source, then we
> > > >
> > > > > trigger a savepoint, and after the source received
> > > >
> > > > > notifyCheckpointComplete, the source would
> > > >
>
> > > > > start emitting EndOfPartitionEvent to finish the job, am I correct ?
> > > >
> > > > >
> > > >
> > > > > For normal finish, a difference to me might be if we have multiple
> > > >
> > > > > sources, we could not guarantee
> > > >
> > > > > when the sources are to finish. We might have one source run one 1
> > > minute
> > > >
> > > > > and another one run for
> > > >
>
> > > > > 1 hour. To unify with the process with stop with savepoint, we might
> > > need
> > > >
> > > > > to hold the fast source until
> > > >
> > > > > all the sources are finished? An coordinator would be introduced to
> > > count
> > > >
> > > > > the number of sources
> > > >
>
> > > > > runing and trigger the final savepoint / checkpoint. For the extreme
> > > >
> > > > > cases, if we have both bounded and
> > > >
> > > > > unbounded sources, we might only count how much bounded source are
> > > >
> > > > > remaining ? And if all the bounded
> > > >
> > > > > sources are finished we would trigger the special checkpoint. After
> > all
> > > >
> > > > > the bounded part of the graph are
> > > >
> > > > > finished, the the remaining part could still do checkpoint and
> > commit
> > > >
> > > > data
> > > >
> > > > > with FLIP-147.
> > > >
> > > > >
> > > >
> > > > > Best,
> > > >
> > > > > Yun
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > ------------------Original Mail ------------------
> > > >
> > > > > Sender:Guowei Ma
> > > >
> > > > > Send Date:Wed Feb 24 17:35:36 2021
> > > >
> > > > > Recipients:dev
> > > >
> > > > > CC:Arvid Heise
> > > >
> > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > > Finished
> > > >
> > > > > Hi, Till
> > > >
> > > > >
> > > >
> > > > > Thank you very much for your careful consideration
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > *1. Emit records in `NotifyCheckpointComplete`.*
> > > >
> > > > >
> > > >
> > > > > Sorry for making you misunderstanding because of my expression. I
> > just
> > > >
> > > > >
> > > >
> > > > > want to say the current interface does not prevent users from doing
> > it.
> > > >
> > > > >
> > > >
> > > > > From the perspective of the new sink api, we might not depend on
> > > emitting
> > > >
> > > > >
> > > >
> > > > > records in `NotifyCheckpointComplete`, like using
> > `OperatorCoordinator`
> > > >
> > > > >
> > > >
> > > > > instead.
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > *2. What does the FLIP-147 guarantee?*I think initially this FLIP
> > want
> > > to
> > > >
> > > > >
> > > >
> > > > > achieve two targets:
> > > >
> > > > >
> > > >
>
> > > > > 1. Tasks/Operators exit correctly (as you mentioned the lifecycle of
> > a
> > > >
> > > > >
> > > >
> > > > > Task/StreamTask/StreamOperator.).
> > > >
> > > > >
> > > >
> > > > > 2. Continue to trigger checkpoint after some tasks for mixed jobs.
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
>
> > > > > I think the first thing is related to the discussion in FLINK-21133.
> > > If I
> > > >
> > > > >
> > > >
> > > > > understand correctly, in addition to supporting the tasks /
> > operators
> > > to
> > > >
> > > > >
> > > >
> > > > > exit correctly, now we also want to unify the process of the tasks
> > and
> > > >
> > > > >
> > > >
> > > > > operators for savepoint / finish.
> > > >
> > > > >
> > > >
> > > > > I think the second thing is orthogonal to the FLINK-21133 because
> > there
> > > >
> > > > are
> > > >
> > > > >
> > > >
> > > > > topologies that have both the bounded and unbounded input.
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > *3. How to unify the operator exit process of FLIP-147 with
> > > >
> > > > >
> > > >
> > > > > stop-with-savepoint?*
> > > >
> > > > >
> > > >
> > > > > I am not very sure about how to do it yet. But if I understand the
> > > >
> > > > >
> > > >
> > > > > discussion in the jira correctly it needs to introduce some logic
> > into
> > > >
> > > > >
> > > >
>
> > > > > `CheckpointCoordinator`, which responses for triggering “the unified
> > > >
> > > > >
> > > >
> > > > > operator exit process”. Am I correct?
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > Best,
> > > >
> > > > >
> > > >
> > > > > Guowei
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann wrote:
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > > Thanks for the explanation Yun and Guowei. I have to admit that I
> > do
> > > >
> > > > not
> > > >
> > > > >
> > > >
>
> > > > > > fully understand why this is strictly required but I think that we
> > > are
> > > >
> > > > >
> > > >
> > > > > > touching two very important aspects which might have far fetching
> > > >
> > > > >
> > > >
> > > > > > consequences for how Flink works:
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > 1) Do we want to allow that multiple checkpoints are required to
> > > >
> > > > >
> > > >
> > > > > > materialize results?
> > > >
> > > > >
> > > >
> > > > > > 2) Do we want to allow to emit records in
> > notifyCheckpointComplete?
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > For 1) I am not sure whether this has been discussed within the
> > > >
> > > > community
> > > >
> > > > >
> > > >
> > > > > > sufficiently. Requiring multiple checkpoints to materialize a
> > result
> > > >
> > > > >
> > > >
> > > > > > because of multi level committers has the consequence that we
> > > increase
> > > >
> > > > > the
> > > >
> > > > >
> > > >
>
> > > > > > latency from checkpoint interval to #levels * checkpoint interval.
> > > >
> > > > >
> > > >
> > > > > > Moreover, having to drain the pipeline in multiple steps, would
> > break
> > > >
> > > > the
> > > >
> > > > >
> > > >
> > > > > > stop-with-savepoint --drain because which savepoint do you report
> > to
> > > >
> > > > the
> > > >
> > > > >
> > > >
> > > > > > user?
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > For 2) allowing to send records after the final
> > > >
> > > > notifyCheckpointComplete
> > > >
> > > > >
> > > >
> > > > > > will effectively mean that we need to shut down a topology in
> > > multiple
> > > >
> > > > >
> > > >
> > > > > > steps (in the worst case one operator per checkpoint). This would
> > be
> > > a
> > > >
> > > > >
> > > >
> > > > > > strong argument for not allowing this to me. The fact that users
> > can
> > > >
> > > > send
> > > >
> > > > >
> > > >
> > > > > > records after the notifyCheckpointComplete is more by accident
> > than
> > > by
> > > >
> > > > >
> > > >
>
> > > > > > design. I think we should make this a very deliberate decision and
> > in
> > > >
> > > > > doubt
> > > >
> > > > >
> > > >
>
> > > > > > I would be in favour of a more restrictive model unless there is a
> > > very
> > > >
> > > > >
> > > >
> > > > > > good reason why this should be supported.
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > Taking also the discussion in FLINK-21133 [1] into account, it
> > seems
> > > to
> > > >
> > > > > me
> > > >
> > > > >
> > > >
> > > > > > that we haven't really understood what kind of guarantees we want
> > to
> > > >
> > > > give
> > > >
> > > > >
> > > >
> > > > > > to our users and how the final checkpoint should exactly work. I
> > > >
> > > > > understand
> > > >
> > > > >
> > > >
> > > > > > that this is not included in the first scope of FLIP-147 but I
> > think
> > > >
> > > > this
> > > >
> > > > >
> > > >
> > > > > > is so important that we should figure this out asap. Also because
> > the
> > > >
> > > > > exact
> > > >
> > > > >
> > > >
> > > > > > shut down behaviour will have to be aligned with the lifecycle of
> > a
> > > >
> > > > >
> > > >
> > > > > > Task/StreamTask/StreamOperator. And last but not least because
> > other
> > > >
> > > > >
> > > >
> > > > > > features such as the new sink API start building upon a shut down
> > > model
> > > >
> > > > >
> > > >
> > > > > > which has not been fully understood/agreed upon.
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > [1] https://issues.apache.org/jira/browse/FLINK-21133
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > Cheers,
> > > >
> > > > >
> > > >
> > > > > > Till
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma wrote:
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > > > > Thanks Yun for the detailed explanation.
> > > >
> > > > >
> > > >
>
> > > > > > > A simple supplementary explanation about the sink case: Maybe we
> > > >
> > > > could
> > > >
> > > > >
> > > >
> > > > > > use
> > > >
> > > > >
> > > >
> > > > > > > `OperatorCoordinator` to avoid sending the element to the
> > > downstream
> > > >
> > > > >
> > > >
> > > > > > > operator.
> > > >
> > > > >
> > > >
> > > > > > > But I agree we could not limit the users not to emit records in
> > the
> > > >
> > > > >
> > > >
> > > > > > > `notiyCheckpointComplete`.
> > > >
> > > > >
> > > >
> > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > Best,
> > > >
> > > > >
> > > >
> > > > > > > Guowei
> > > >
> > > > >
> > > >
> > > > > > >
> > > >
> > > > >
> > > >
> > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao
> > > >
> > > > >
> > > >
> > > > > > > wrote:
> > > >
> > > > >
> > > >
> > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > Hi all,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > I'd like to first detail the issue with emitting records in
> > > >
> > > > >
> > > >
> > > > > > > > notifyCheckpointComplete for context. For specific usage,
> > > >
> > > > >
> > > >
> > > > > > > > an example would be for sink, it might want to write some
> > > metadata
> > > >
> > > > >
> > > >
> > > > > > after
> > > >
> > > > >
> > > >
> > > > > > > > all the transactions are committed
> > > >
> > > > >
> > > >
> > > > > > > > (like write a marker file _SUCCESS to the output directory).
> > This
> > > >
> > > > > case
> > > >
> > > > >
> > > >
> > > > > > is
> > > >
> > > > >
> > > >
> > > > > > > > currently supported via the two level
> > > >
> > > > >
> > > >
> > > > > > > > committers of the new sink API: when received endOfInput(),
> > the
> > > >
> > > > >
> > > >
> > > > > > Committer
> > > >
> > > > >
> > > >
> > > > > > > > wait for another checkpoint to
> > > >
> > > > >
> > > >
> > > > > > > > commits all the pending transactions and emit the list of
> > files
> > > to
> > > >
> > > > > the
> > > >
> > > > >
> > > >
> > > > > > > > GlobalCommitter. The GlobalCommitter
> > > >
> > > > >
> > > >
> > > > > > > > would wait for another checkpoint to also write the metadata
> > with
> > > >
> > > > 2pc
> > > >
> > > > >
> > > >
> > > > > > > > (Although sometimes 2pc is not needed
> > > >
> > > > >
> > > >
> > > > > > > > for writing metadata, it should be only an optimization and
> > still
> > > >
> > > > >
> > > >
> > > > > > > requires
> > > >
> > > > >
> > > >
> > > > > > > > the Committer do commit before
> > > >
> > > > >
> > > >
> > > > > > > > notifying the global Committer. Also another note is
> > > >
> > > > GlobalCommitter
> > > >
> > > > > is
> > > >
> > > > >
> > > >
> > > > > > > > also added for some other cases
> > > >
> > > > >
> > > >
> > > > > > > > like some sinks want an commiter with dop = 1, like
> > IceBergSink).
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > However, a more general issue to me is that currently we do
> > not
> > > >
> > > > limit
> > > >
> > > > >
> > > >
> > > > > > > > users to not emit records in
> > > >
> > > > >
> > > >
>
> > > > > > > > notifyCheckpointComplete in the API level. The sink case could
> > be
> > > >
> > > > >
> > > >
> > > > > > viewed
> > > >
> > > > >
> > > >
> > > > > > > > as a special case, but in addition
> > > >
> > > > >
> > > >
> > > > > > > > to this one, logically users could also implement their own
> > cases
> > > >
> > > > > that
> > > >
> > > > >
> > > >
> > > > > > > > emits records in notifyCheckpointComplete.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > Best,
> > > >
> > > > >
> > > >
> > > > > > > > Yun
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > ------------------Original Mail ------------------
> > > >
> > > > >
> > > >
> > > > > > > > Sender:Arvid Heise
> > > >
> > > > >
> > > >
> > > > > > > > Send Date:Fri Feb 12 20:46:04 2021
> > > >
> > > > >
> > > >
> > > > > > > > Recipients:dev
> > > >
> > > > >
> > > >
> > > > > > > > CC:Yun Gao
> > > >
> > > > >
> > > >
> > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After
> > Tasks
> > > >
> > > > >
> > > >
> > > > > > Finished
> > > >
> > > > >
> > > >
> > > > > > > > Hi Piotr,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > Thank you for raising your concern. Unfortunately, I do not
> > have
> > > a
> > > >
> > > > >
> > > >
> > > > > > better
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > idea than doing closing of operators intermittently with
> > > >
> > > > checkpoints
> > > >
> > > > > (=
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > multiple last checkpoints).
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > However, two ideas on how to improve the overall user
> > experience:
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > 1. If an operator is not relying on notifyCheckpointComplete,
> > we
> > > >
> > > > can
> > > >
> > > > >
> > > >
> > > > > > > close
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > it faster (without waiting for a checkpoint). In general, I'd
> > > >
> > > > assume
> > > >
> > > > >
> > > >
> > > > > > that
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > almost all non-sinks behave that way.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > 2. We may increase the checkpointing frequency for the last
> > > >
> > > > >
> > > >
> > > > > > checkpoints.
> > > >
> > > > >
> > > >
> > > > > > > We
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > need to avoid overloading checkpoint storages and task
> > managers,
> > > >
> > > > but
> > > >
> > > > > I
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > assume the more operators are closed, the lower the
> > checkpointing
> > > >
> > > > >
> > > >
> > > > > > > interval
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > can be.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > For 1, I'd propose to add (name TBD):
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > default boolean StreamOperator#requiresFinalCheckpoint() {
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > return true;
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > }
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > This means all operators are conservatively (=slowly) closed.
> > For
> > > >
> > > > > most
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > operators, we can then define their behavior by overriding in
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > AbstractUdfStreamOperator
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > @Override
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > return userFunction instanceof CheckpointListener;
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > }
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > This idea can be further refined in also adding
> > > >
> > > > > requiresFinalCheckpoint
> > > >
> > > > >
> > > >
> > > > > > > to
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > CheckpointListener to exclude all operators with UDFs that
> > > >
> > > > implement
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > CheckpointListener but do not need it for 2pc.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > @Override
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > return userFunction instanceof CheckpointListener &&
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > ((CheckpointListener) userFunction).requiresFinalCheckpoint();
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > }
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > That approach would also work for statebackends/snapshot
> > > strategies
> > > >
> > > > >
> > > >
> > > > > > that
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > require some 2pc.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > If we can contain it to the @PublicEvolving StreamOperator, it
> > > >
> > > > would
> > > >
> > > > > be
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > better of course.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > Best,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > Arvid
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > wrote:
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > Hey,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > I would like to raise a concern about implementation of the
> > > final
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > checkpoints taking into account operators/functions that are
> > > >
> > > > >
> > > >
> > > > > > > implementing
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > two phase commit (2pc) protocol for exactly-once processing
> > > with
> > > >
> > > > > some
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > external state (kept outside of the Flink). Primarily
> > > >
> > > > exactly-once
> > > >
> > > > >
> > > >
> > > > > > > sinks.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > First of all, as I understand it, this is not planned in the
> > > >
> > > > first
> > > >
> > > > >
> > > >
> > > > > > > > version
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > of this FLIP. I'm fine with that, however I would strongly
> > > >
> > > > > emphasize
> > > >
> > > > >
> > > >
> > > > > > > this
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > in every place we will be mentioning FLIP-147 efforts. This
> > is
> > > >
> > > > >
> > > >
> > > > > > because
> > > >
> > > > >
> > > >
> > > > > > > > me,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > as a user, upon hearing "Flink supports checkpointing with
> > > >
> > > > bounded
> > > >
> > > > >
> > > >
> > > > > > > > inputs"
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > I would expect 2pc to work properly and to commit the
> > external
> > > >
> > > > side
> > > >
> > > > >
> > > >
> > > > > > > > effects
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > upon finishing. As it is now, I (as a user) would be
> > surprised
> > > >
> > > > > with a
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > silent data loss (of not committed trailing data). This is
> > > just a
> > > >
> > > > >
> > > >
> > > > > > > remark,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > that we need to attach this warning to every blog
> > > >
> > > > >
> > > >
> > > > > > > post/documentation/user
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > mailing list response related to "Support Checkpoints After
> > > Tasks
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > Finished". Also I would suggest to prioritize the follow up
> > of
> > > >
> > > > >
> > > >
> > > > > > > supporting
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 2pc.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > Secondly, I think we are missing how difficult and
> > problematic
> > > >
> > > > will
> > > >
> > > > >
> > > >
> > > > > > be
> > > >
> > > > >
> > > >
> > > > > > > > 2pc
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > support with the final checkpoint.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > For starters, keep in mind that currently 2pc can be
> > > implemented
> > > >
> > > > by
> > > >
> > > > >
> > > >
> > > > > > > users
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > using both `@Public` APIs as functions and `@PublicEvolving`
> > > >
> > > > >
> > > >
> > > > > > operators
> > > >
> > > > >
> > > >
> > > > > > > in
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > any place in the job graph. It's not limited to only the
> > sinks.
> > > >
> > > > For
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > example users could easily implement the `AsynFunction` (for
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > `AsyncWaitOperator`) that is using 2pc based on the
> > > >
> > > > >
> > > >
> > > > > > > `CheckpointListener`
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > interface. I'm not saying it's common, probably just a tiny
> > > >
> > > > > minority
> > > >
> > > > >
> > > >
> > > > > > of
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > users are doing that (if any at all), but nevertheless
> > that's
> > > >
> > > > >
> > > >
> > > > > > possible
> > > >
> > > > >
> > > >
> > > > > > > > and
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > currently (implicitly?) supported in Flink.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > Next complication is the support of bounded streams
> > > >
> > > > >
> > > >
> > > > > > (`BoundedOneInput`
> > > >
> > > > >
> > > >
> > > > > > > or
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > `BoundedMultiInput` interfaces) and the closing/shutdown
> > > >
> > > > procedure
> > > >
> > > > > of
> > > >
> > > > >
> > > >
> > > > > > > the
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > operators. Currently it works as follows:
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > 1. `endOfInput` is called on the first operator in the chain
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 2. We quiesce the processing timers
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> > > for
> > > >
> > > > > the
> > > >
> > > > >
> > > >
> > > > > > > > first
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > operator, so no new timers will be triggered
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 3. We wait for the already fired timers to finish executing
> > > >
> > > > > (spinning
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > mailbox loop)
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 4. We are closing the first operator
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 5. We go to the next (second) operator in the chain and
> > repeat
> > > >
> > > > the
> > > >
> > > > >
> > > >
> > > > > > > steps
> > > >
> > > > >
> > > >
> > > > > > > > 1.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > to 5.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > This is because operators can emit data after processing
> > > >
> > > > >
> > > >
> > > > > > `endOfInput`,
> > > >
> > > > >
> > > >
> > > > > > > > from
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > timers, async mailbox actions and inside the `close` method
> > > >
> > > > itself.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > Now the problem is to support the final checkpoint with 2pc,
> > we
> > > >
> > > > > need
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > trigger `snapshotState` and `notifyCheckpointComplete` call
> > at
> > > >
> > > > the
> > > >
> > > > >
> > > >
> > > > > > very
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > least only after `endOfInput` call on the operator. Probably
> > > the
> > > >
> > > > > best
> > > >
> > > > >
> > > >
> > > > > > > > place
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > would be in between steps 3. and 4. However that means, we
> > > would
> > > >
> > > > be
> > > >
> > > > >
> > > >
> > > > > > > > forced
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > to wait for steps 1. to 3. to finish, then wait for a next
> > > >
> > > > > checkpoint
> > > >
> > > > >
> > > >
> > > > > > > to
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > trigger AND complete, before finally closing the head
> > operator,
> > > >
> > > > and
> > > >
> > > > >
> > > >
> > > > > > > only
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > then we can start closing the next operator in the chain:
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > 1. `endOfInput` is called on the first operator in the chain
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 2. We quiesce the processing timers
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> > > for
> > > >
> > > > > the
> > > >
> > > > >
> > > >
> > > > > > > > first
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > operator, so no new timers will be triggered
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 3. We wait for the already fired timers to finish executing
> > > >
> > > > > (spinning
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > mailbox loop)
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > *3b. We wait for one more checkpoint to trigger and for the
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > `notifyCheckpointComplete` RPC.*
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 4. We are closing the first operator
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 5. We go to the next (second) operator in the chain and
> > repeat
> > > >
> > > > the
> > > >
> > > > >
> > > >
> > > > > > > steps
> > > >
> > > > >
> > > >
> > > > > > > > 1.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > to 5.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > That means, we can close one operator per successful
> > > checkpoint.
> > > >
> > > > To
> > > >
> > > > >
> > > >
> > > > > > > close
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > 10 operators, we would need 10 successful checkpoints.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > I was thinking about different approaches to this problem,
> > and
> > > I
> > > >
> > > > >
> > > >
> > > > > > > couldn't
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > find any viable ones. All I could think of would break the
> > > >
> > > > current
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > `@Public` API and/or would be ugly/confusing for the users.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > For example a relatively simple solution, to introduce a
> > > >
> > > > `preClose`
> > > >
> > > > >
> > > >
> > > > > > or
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > `flush` method to the operators, with a contract that after
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > `flush`, operators would be forbidden from emitting more
> > > records,
> > > >
> > > > > so
> > > >
> > > > >
> > > >
> > > > > > > that
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > we can replace step 4. with this `flush` call, and then
> > having
> > > a
> > > >
> > > > >
> > > >
> > > > > > single
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > checkpoint to finish 2pc for all of the operators inside the
> > > >
> > > > chain,
> > > >
> > > > >
> > > >
> > > > > > > > doesn't
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > work. Sheer fact of adding this `flush` method and changing
> > the
> > > >
> > > > >
> > > >
> > > > > > > contract
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > would break the current API and Yun Gao has pointed out to
> > me,
> > > >
> > > > that
> > > >
> > > > >
> > > >
> > > > > > we
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > either already support, or want to support operators that
> > are
> > > >
> > > > >
> > > >
> > > > > > emitting
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > records from within the `notifyCheckpointComplete` call:
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > Yun Gao:
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > like with the new sink api there might be writer ->
> > committer
> > > >
> > > > ->
> > > >
> > > > >
> > > >
> > > > > > > global
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > committer, the committer would need to wait for the last
> > > >
> > > > checkpoint
> > > >
> > > > >
> > > >
> > > > > > to
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > commit
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > the last piece of data, and after that it also need to
> > emit
> > > the
> > > >
> > > > >
> > > >
> > > > > > list
> > > >
> > > > >
> > > >
> > > > > > > of
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > transactions get committed to global committer to do some
> > > >
> > > > >
> > > >
> > > > > > finalization
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > logic.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > So it wouldn't solve the problem (at least not fully).
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > I don't know if anyone has any better ideas how to solve
> > this
> > > >
> > > > >
> > > >
> > > > > > problem?
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > Piotrek
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > pt., 15 sty 2021 o 14:57 Yun Gao
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > napisał(a):
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > Hi Aljoscha,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > I think so since we seems to do not have other divergence
> > and
> > > >
> > > > new
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > objections now. I'll open the vote then. Very thanks!
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > Best,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > Yun
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > > ------------------------------------------------------------------
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > From:Aljoscha Krettek
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > Send Time:2021 Jan. 15 (Fri.) 21:24
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > To:dev
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After
> > > Tasks
> > > >
> > > > >
> > > >
> > > > > > > > Finished
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > Thanks for the summary! I think we can now move towards a
> > > >
> > > > [VOTE]
> > > >
> > > > >
> > > >
> > > > > > > > thread,
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > right?
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > On 2021/01/15 13:43, Yun Gao wrote:
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > >1) For the problem that the "new" root task coincidently
> > > >
> > > > > finished
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > > >before getting triggered successfully, we have listed two
> > > >
> > > > > options
> > > >
> > > > >
> > > >
> > > > > > in
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > >the FLIP-147[1], for the first version, now we are not
> > tend
> > > to
> > > >
> > > > > go
> > > >
> > > > >
> > > >
> > > > > > > with
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > >the first option that JM would re-compute and re-trigger
> > new
> > > >
> > > > >
> > > >
> > > > > > sources
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > >when it realized some tasks are not triggered
> > successfully.
> > > >
> > > > This
> > > >
> > > > >
> > > >
> > > > > > > > option
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > >would avoid the complexity of adding new PRC and
> > duplicating
> > > >
> > > > > task
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > >states, and in average case it would not cause too much
> > > >
> > > > > overhead.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
>
> > > > > > > > > > You wrote "we are *not* tend to go with the first option",
> > > but
> > > >
> > > > I
> > > >
> > > > >
> > > >
> > > > > > > think
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > you meant wo write "we tend to *now* go with the first
> > > option",
> > > >
> > > > >
> > > >
> > > > > > > right?
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > That's also how it is in the FLIP, I just wanted to
> > clarify
> > > for
> > > >
> > > > > the
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > > mailing list.
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > > >
> > > >
> > > > >
> > > >
> > > > > > >
> > > >
> > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > >
> > >
> >
> >
>
>
>

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
(Sorry that I repeat this mail since the last one is not added into the same mail list thread,
very sorry for the inconvenience)

 Hi all,

    Very thanks for all the deep thoughts!

> How to implement the stop-with-savepoint --drain/terminate command with
> this model: One idea could be to tell the sources that they should stop
> reading. This should trigger the EndOfPartitionEvent to be sent
> downstream.
> This will transition all operators into the TERMINATING state.
>
> Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
> returned. To achieve above, possible works should be required:
> * Promote  `EndOfPartitionEvent` from `Task` to `StreamTask`. This may
> have some interferences with BatchTask or network io stack.
> * Or introducing stream task level `EndOfUserRecordsEvent`(from PR#14831
> @Yun @Piotr)
> * Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.

I also have similar concern with Kezhu for the issue whether do we need to introduce a new message
to notify the operators to endOfInput/close ? The main concerns of reusing that EndOfPartitionEvent
is that
1. The EndOfPartitionEvent is currently emitted in Task instead of StreamTask, we would need some
refactors here.
2. Currently the InputGate/InputChannel would be released after the downstream tasks have received 
EndOfPartitionEvent from all the input channels, this would makes the following checkpoint unable to 
perform since we could not emit barriers to downstream tasks ?

Regarding the MAX_WATERMARK, I still not fully understand the issue since it seems to me
that now Flink won't snapshot the watermark now? If the job failover, the window operator
would reload all the pending windows before flushed by MAX_WATERMARK and when the
job finish again, it would re-emit the MAX_WATERMARK?

Best,
Yun



------------------------------------------------------------------
From:Kezhu Wang <ke...@gmail.com>
Send Time:2021 Mar. 1 (Mon.) 01:26
To:Till Rohrmann <tr...@apache.org>
Cc:Piotr Nowojski <pi...@gmail.com>; Guowei Ma <gu...@gmail.com>; dev <de...@flink.apache.org>; Yun Gao <yu...@aliyun.com>; jingsonglee0@gmail.com <ji...@gmail.com>
Subject:Re: Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

In “stop-with-savepoint —drain”, MAX_WATERMARK is not an issue. For normal
finishing task, not allowing unaligned checkpoint does not solve the
problem as MAX_WATERMARK could be persisted in downstream task. When
scenario @Piotr depicted occurs, downstream(or further downstream) window
operator will count all inputs as late.

> If we ensure that the MAX_WATERMARK is only persisted in state if a
recovery will trigger immediately the shut down of this operator, then it
shouldn't be an issue.

You are right in case the assumption holds, I have same thought as you
before. But I am kind of worry about whether it is too prefect to be
fragile. This requires strong guarantee from implementation that a recovery
from TERMINATING stage should go directly to that stage.

> I think the MAX_WATERMARK event should be sent either just before or with
the EndOfPartitionEvent.

I prefer “with the EndOfPartitionEvent”. EndOfPartitionEvent itself already
carry what ending MAX_WATERMARK try to express. May be we can reuse it ?
@Piotr

A preview work[1] from @Yun in PR#14831 explains EndOfPartitionEvent as
checkpoint barrier if there are pending checkpoints.


[1]:
https://github.com/gaoyunhaii/flink/commit/1aaa80fb0afad13a314b06783c61c01b9414f91b#diff-d4568b34060bc589cd1354bd125c35aca993dd0c9fe9e4460dfed73501116459R177


Best,
Kezhu Wang

On February 28, 2021 at 21:23:31, Till Rohrmann (trohrmann@apache.org)
wrote:

I think you are right with the problem of endOfInput. endOfInput should not
be used to commit final results. In fact if this termination fails then we
might end up in a different outcome of the job which is equally valid as
the one before the failure.

Concerning unaligned checkpoints, I think they don't play well together
with draining a streaming pipeline. The problem is that in the draining
case you want to process all records which are still in flight but
unaligned checkpoints don't guarantee this as they can jump in flight
records.

I think the MAX_WATERMARK event should be sent either just before or with
the EndOfPartitionEvent. If we ensure that the MAX_WATERMARK is only
persisted in state if a recovery will trigger immediately the shut down of
this operator, then it shouldn't be an issue.

Cheers,
Till

On Sun, Feb 28, 2021 at 9:06 AM Kezhu Wang <ke...@gmail.com> wrote:

> Hi Till,
>
> Just for bookkeeping, some observations from current implementation.
>
> > With this model, the final checkpoint is quite simple because it is
> ingrained in the lifecycle of an operator. Differently said an operator
> will only terminate after it has committed its side effects and seen the
> notifyCheckpointComplete message (if it is stateful).
>
> Currently, we could not mark this operator(or subtask) as terminated since
> result of `notifyCheckpointComplete`(possible side effect committing) is
> not taken into account of the belonging checkpoint. The job has to run to
> next safe point(finished or next checkpoint success) to be marked as
> “terminated”.
>
> > How to implement the stop-with-savepoint --drain/terminate command with
> this model: One idea could be to tell the sources that they should stop
> reading. This should trigger the EndOfPartitionEvent to be sent
> downstream.
> This will transition all operators into the TERMINATING state.
>
> Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
> returned. To achieve above, possible works should be required:
> * Promote  `EndOfPartitionEvent` from `Task` to `StreamTask`. This may
> have some interferences with BatchTask or network io stack.
> * Or introducing stream task level `EndOfUserRecordsEvent`(from PR#14831
> @Yun @Piotr)
> * Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.
>
> Besides this, I would like to quote some discussion from FLINK-21467
> between @Piotr and me:
>
> From @Piotr
> > Note, that it's not only that endOfInput can be called multiple times.
> There is a very remote possibility that the following scenario will happen:
> 1. checkpoint is taken (successfully)
> 2. sources are finishing
> 3. endOfInput is issued
> 4. job fails
> 5. job restarts to checkpoint 1.
> 6. after failover, because of some non deterministic logic in the source,
> sources are not finishing
>
> From me
> > But I think there is little work Flink can do to cope with this kind of
> issues. The checkpoint could be a savepoint triggered from user side and
> the "non deterministic logic" could be a change from user(eg. changing of
> stoppingOffsets in KafkaSource).
>
> > I think the "non deterministic logic" could cause trouble in combination
> with unaligned checkpoint and downstream window operator. Unaligned
> checkpoint will persist "MAX_WATERMARK" in state, after restarting,
> "MAX_WATERMARK" will cause downstream window operator ignores all future
> inputs.
>
> FLIP-147 demands no new records from end-of-stream-flushing, but source
> will emit “MAX_WATERMARK” on ending. Previously, I thought it is not a
> valid issue, but turn out that it could cause trouble under scenario listed
> by @Piotr if I am not wrong.
>
>
> PR#14831: https://github.com/apache/flink/pull/14831
> FLINK-21467: https://issues.apache.org/jira/browse/FLINK-21467
>
>
> Best,
> Kezhu Wang
>
> On February 27, 2021 at 18:12:20, Till Rohrmann (trohrmann@apache.org)
> wrote:
>
> Thanks for all your thoughts. I think we should further think through
> whether to allow checkpoints after an operator has emitted all its records
> (e.g. after close is called currently) or not. I think by doing this we
> would nicely decouple the checkpoint taking from the operator lifecycle
> and
> wouldn't need special checkpoints/savepoints for the final checkpoint and
> stop-with-savepoint --drain. Let me try to explain this a bit more
> detailed.
>
> If we say an operator has the rough lifecycle RUNNING => TERMINATING =>
> TERMINATED where we go from RUNNING into TERMINATING after we have seen
> the
> EndOfPartitionEvent and flushed all our records. The operator goes from
> TERMINATING => TERMINATED if it has persisted all its possible side
> effects. Throughout all states, it is possible to trigger a checkpoint. A
> stateless operator will immediately go from TERMINATING to TERMINATED
> whereas a stateful operator would wait for another checkpoint to be
> triggered and successfully completed (notifyCheckpointComplete).
>
> With this model, the final checkpoint is quite simple because it is
> ingrained in the lifecycle of an operator. Differently said an operator
> will only terminate after it has committed its side effects and seen the
> notifyCheckpointComplete message (if it is stateful). Here it is important
> to note that in the streaming case, different bounded operators can
> terminate at different times. They don't have to terminate all with the
> same checkpoint.
>
> How to implement the stop-with-savepoint --drain/terminate command with
> this model: One idea could be to tell the sources that they should stop
> reading. This should trigger the EndOfPartitionEvent to be sent
> downstream.
> This will transition all operators into the TERMINATING state. Next the JM
> can trigger a checkpoint to shut the operators down and not to wait for
> the
> next automatic checkpoint trigger event.
>
> By allowing checkpoints throughout the entire lifecycle of an operator we
> disallow sending records from notifyCheckpointComplete because this
> message
> will also be sent in the state TERMINATING where an operator has already
> produced all of its records.
>
> What do you think? Maybe this model overlooks some important cases. One
> downside is that we will break the operator API with changing the
> lifecycle
> of an operator.
>
> Cheers,
> Till
>
>
>
> On Fri, Feb 26, 2021 at 7:29 AM Kezhu Wang <ke...@gmail.com> wrote:
>
> > Hi Yun,
> >
> > The termination phase I drafted depends on a de-fact that
> > `notifyCheckpointComplete` is ignored currently after `close`. Thus, any
> > external materialization inside `close`(including `endInput`) or
> > `notifyCheckpointComplete` is either not guaranteed or not committed.
> >
> > I tried to emphasize no-breaking changes and post-pone migration in
> later
> > releases for this termination phase. But now, I am kind of worry about
> > whether it will cause long-term maintenance hell.
> >
> > Personally, I think allowing checkpoint after `close`(@Till proposed
> this
> > in FLINK-21133) could be the minimal change. But there are concerns:
> > * It will break some existing code possibly in a silent way.(@Piotr
> pointed
> > this already in FLINK-21133)
> > * I think the name `close` is kind of misleading. (@Piotr suggested
> > renaming this to `finish` to not break code silently in FLINK-21133)
> >
> > > stop-with-savepoint --drain would wait for a specific savepoint
> >
> > For stop-with-savepoint, the checkpoint barrier is already created or
> > received there.
> >
> >
> > Best,
> > Kezhu Wang
> >
> > On February 25, 2021 at 21:02:30, Yun Gao (yungao.gy@aliyun.com) wrote:
> >
> > Hi all,
> >
> > Very thanks for the discussions!
> >
> >
> >
> > A. Regarding how to avoid emitting records in notifyCheckpointComplete:
> >
> > Currently the structure of a new sink is writer -> committer -> global
> > committer and the paralellism of global committer
> > must be one. By design it would be used in several cases:
> > 1. writer -> committer: for normal sinks that write pending data in
> writer
> > and commit pending data in committer.
> > 2. writer -> global committer: for sinks require the committer's
> > parallelism be 1, like iceberg sink which
> > relies on optimistic lock to commit data hopes to reduce the conflicts.
> > 3. writer -> committer -> global committer: for sinks that also need to
> > write meta data (like _SUCCESS file
> > or add record in hive metastore) in global committer.
> >
> > The third case would cause the cascade commit problem. To overcome this
> > problem, we would like to
> > 1. Previously the global committer also support writing metadata with
> 2pc.
> > Now we disable this mode
> > and always rely on the property that writing metadata is repeatable.
> With
> > this limitation there should no
> > need of cascade commit, but the committer would still emit records in
> > notifyCheckpointComplete.
> > 2. We further move global committer in the case 3 to the operator
> > coordinator. Then the committer would
> > not need to emit records, but communicates with its operator
> coordinator.
> >
> > One core issue for using the OperatorCoordinator is how to keep the
> > communication between the operator
> > and the OperatorCoordinator exactly-once. Since the message is always
> from
> > the oeprator to the OperatorCoordinator
> > in this case, we would only need to bookkeep the message sent between
> the
> > OperatorCoordinator takes snapshot and
> > the Operator takes snapshot in the state of the Operator.
> >
> > On how to achieve the change in detail we would still need some think,
> it
> > currently seems we would have to had
> > some modification to the current new sink api.
> >
> >
> >
> > B. Regarding the stop-with-savepoint --drain
> >
> > Very thanks @Piotr for the further explanation and now I realize I have
> > understand wrongly for the semantics of
> > stop-with-savepoint --drain. Now I think that the problem should be we
> > should also include the records produced in
> > `endOfInput()` and `close()` also in the last savepoint, am I correct?
> If
> > so, it seems we still have some undetermined options for
> > the lifecycle of the operator, like in Kezhu's proposal the close()
> happens
> > at last, but it seems close() might also emit records (
> > so now the operator are closed with op1's close() -> op2's endOfInput()
> ->
> > op2's close() -> op3's endOfinput -> ...) ?
> >
> > And on the other side, as Kezhu has also proposed, perhapse we might
> have
> > the stop-with-savepoint --drain and normal exit in the same process,
> > but have slightly difference in that stop-with-savepoint --drain would
> wait
> > for a specific savepoint and in normal exit, the operator
> > might wait for arbitrary checkpoint. If we could achieve not emitting
> > records in notifyCheckpointComplete, stop-with-savepoint --drain could
> > be done with one savepoint, and for the normal exit, the operator would
> not
> > need to wait for other slow operators to exit.
> >
> > Best,
> > Yun
> >
> >
> >
> > ------------------Original Mail ------------------
> > *Sender:*Kezhu Wang <ke...@gmail.com>
> > *Send Date:*Thu Feb 25 15:11:53 2021
> > *Recipients:*Flink Dev <de...@flink.apache.org>, Piotr Nowojski <
> > piotr.nowojski@gmail.com>
> > *CC:*Guowei Ma <gu...@gmail.com>, jingsonglee0@gmail.com <
> > jingsonglee0@gmail.com>
> > *Subject:*Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > Finished
> >
> > > Hi all, thanks for driving this and especially Piotr for re-active
> this
> > >
> > > thread.
> > >
> > >
> > >
> > > First, for `notifyCheckpointComplete`, I have strong preference
> towards
> > >
> > > "shut down the dataflow
> > >
> > > pipeline with one checkpoint in total", so I tend to option dropping
> > "send
> > >
> > > records" from
> > >
> > > `notifyCheckpointComplete` for next level committing in pipeline, if
> we
> > >
> > > ever support it. Without this,
> > >
> > > we are unable to stop a pipeline manually with all results
> materialized.
> > >
> > >
> > >
> > > Second, for shutdown unification of `stop-with-savepoint --drain` and
> > >
> > > FLIP-147, I draft following
> > >
> > > phase based on emerging proposals(mainly by Piotr and Till) in this
> > thread
> > >
> > > and FLINK-21133.
> > >
> > >
> > >
> > > ```java
> > >
> > > // StreamTask termination phase
> > >
> > > finish()(Call StreamOperator.finish in chaining order)
> > >
> > > advanceToEndOfEventTime()(nop if for no-source inputs, this could also
> be
> > >
> > > done in input processor or finish ?)
> > >
> > > if (there-is-a-pending-terminate-savepoint) {
> > >
> > > triggerBarrierForDownStream();
> > >
> > > waitCheckpointComplete();
> > >
> > > } else if (require-two-phase-commit-in-shutdown) {
> > >
> > > waitFinalCheckpoint();
> > >
> > > waitCheckpointComplete();
> > >
> > > }
> > >
> > > // TODO: Ignore/Cancel all checkpoints from this point(or hit to
> > >
> > > coordinator in final checkpoint response ?).
> > >
> > > close();
> > >
> > > ```
> > >
> > >
> > >
> > > Branches in `if` could reside in different code paths, but the effect
> > >
> > > should be similar to above.
> > >
> > > The divergence reflects fact that "stop-with-savepoint --drain" is
> > >
> > > triggered already while we need
> > >
> > > to wait for final checkpoint in natural input exhausting.
> > >
> > >
> > >
> > > With carefully chosen default functions, we will not break existing
> > >
> > > interfaces.
> > >
> > > * `StreamOperator.finish`
> > >
> > > * `RichFunction.finish`
> > >
> > >
> > >
> > > For existing codes:
> > >
> > > 1. "stop-with-savepoint" with no migration, it should behaves as
> before.
> > >
> > > 2. "require-two-phase-commit-in-shutdown" evaluates to false and no
> > >
> > > migration, it is same as before.
> > >
> > > 3. "require-two-phase-commit-in-shutdown" evaluates to true and no
> > >
> > > migration, records in `close` will
> > >
> > > lose.
> > >
> > >
> > >
> > > For case#3:
> > >
> > > A. No external materialization in `StreamOperator.close`: Nothing to
> > lose.
> > >
> > > B. External materialization only in `StreamOperator.close`: I don't
> think
> > >
> > > Flink ever claimed that there is
> > >
> > > any guarantee for this situation.
> > >
> > > C. External materialization in `notifyCheckpointComplete` from
> > >
> > > `StreamOperator.close`: But due to fact that
> > >
> > > `notifyCheckpointComplete` was ignored after operator
> > >
> > > closed(FLINK-16383), so there will be no external
> > >
> > > materialization to lose.
> > >
> > >
> > >
> > > Then, we could recommend users to migrate possible
> > “end-of-stream-flushing”
> > >
> > > from “close" to “finish”.
> > >
> > >
> > >
> > > Best,
> > >
> > > Kezhu Wang
> > >
> > >
> > >
> > >
> > >
> > > On February 24, 2021 at 23:47:36, Piotr Nowojski (
> > piotr.nowojski@gmail.com
> > > )
> > >
> > > wrote:
> > >
> > >
> > >
> > > Thanks for the reponses Guowei and Yun,
> > >
> > >
> > >
> > > Could you elaborate more/remind me, what does it mean to replace
> emitting
> > >
> > > results from the `notifyCheckpointComplete` with `OperatorCoordinator`
> > >
> > > approach?
> > >
> > >
> > >
> > > About the discussion in FLINK-21133 and how it relates to FLIP-147.
> You
> > are
> > >
> > > right Yun gao, that in case of `stop-with-savepoint --drain` the whole
> > job
> > >
> > > finishes, while in FLIP-147 some of the sources can work for
> arbitrarily
> > >
> > > long time after others have already finished. However from the runtime
> > >
> > > perspective, especially how to close a Task and it's operators, there
> is
> > no
> > >
> > > difference between those two cases. In both cases, we need to end
> input,
> > >
> > > shut down processing time timers and commit external side effects (two
> > >
> > > phase commit support) before exiting the task. That's how the
> discussion
> > >
> > > about the behaviour of "stop-with-savepoint" was connected with
> FLIP-147.
> > >
> > >
> > >
> > > Currently on master, "stop-with-savepoint --drain" drains/flushes
> > buffered
> > >
> > > records and deals correctly with timers, but all of that happens AFTER
> > >
> > > savepoint was completed. So any records flushed from the operators
> during
> > >
> > > endOfInput/close/shutting down processing timers are never committed
> to
> > >
> > > external systems. This is exactly the same problem as the "two phase
> > >
> > > commit" problem of FLIP-147, that should have the same solution and it
> > >
> > > should be solved at the same time. For example if we go with the *very
> > >
> > > undesirable* "one closed operator per one completed
> > checkpoint/savepoint",
> > >
> > > in both cases CheckpointCoordinator, Scheduler and Task would need to
> > keep
> > >
> > > the task alive and keep triggering checkpoints for that task, until
> all
> > >
> > > operators in the operator chain are closed (one closed operator per
> one
> > >
> > > completed checkpoint).
> > >
> > >
> > >
> > > Piotrek
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > > śr., 24 lut 2021 o 11:30 Yun Gao napisał(a):
> > >
> > >
> > >
> > > > Hi Till, Guowei,
> > >
> > > >
> > >
> > > > Very thanks for initiating the disucssion and the deep thoughts!
> > >
> > > >
> > >
> > > > For the notifyCheckpointComplete, I also agree we could try to avoid
> > >
> > > > emitting new records in notifyCheckpointComplete via using
> > >
> > > > OperatorCoordinator
> > >
> > > > for new sink API. Besides, the hive sink might also need some
> > >
> > > modification
> > >
> > > > for it also emits records in notifyCheckpointComplete.
> > >
> > > >
> > >
> > > > For unifying the process of stopping with savepoint and finished due
> to
> > >
> > > > all records
> > >
> > > > are processed, I also agree with that unifying would always be
> better
> > if
> > >
> > > > we could achieve,
> > >
> > > > but I'm still not fully catch up with the implementation: Based on
> the
> > >
> > > > discussion in FLINK-21133,
> > >
> > > > my understanding is that for stopping with savepoint, now we want to
> > >
> > > first
> > >
> > > > stop the source, then we
> > >
> > > > trigger a savepoint, and after the source received
> > >
> > > > notifyCheckpointComplete, the source would
> > >
> > > > start emitting EndOfPartitionEvent to finish the job, am I correct ?
> > >
> > > >
> > >
> > > > For normal finish, a difference to me might be if we have multiple
> > >
> > > > sources, we could not guarantee
> > >
> > > > when the sources are to finish. We might have one source run one 1
> > minute
> > >
> > > > and another one run for
> > >
> > > > 1 hour. To unify with the process with stop with savepoint, we might
> > need
> > >
> > > > to hold the fast source until
> > >
> > > > all the sources are finished? An coordinator would be introduced to
> > count
> > >
> > > > the number of sources
> > >
> > > > runing and trigger the final savepoint / checkpoint. For the extreme
> > >
> > > > cases, if we have both bounded and
> > >
> > > > unbounded sources, we might only count how much bounded source are
> > >
> > > > remaining ? And if all the bounded
> > >
> > > > sources are finished we would trigger the special checkpoint. After
> all
> > >
> > > > the bounded part of the graph are
> > >
> > > > finished, the the remaining part could still do checkpoint and
> commit
> > >
> > > data
> > >
> > > > with FLIP-147.
> > >
> > > >
> > >
> > > > Best,
> > >
> > > > Yun
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > ------------------Original Mail ------------------
> > >
> > > > Sender:Guowei Ma
> > >
> > > > Send Date:Wed Feb 24 17:35:36 2021
> > >
> > > > Recipients:dev
> > >
> > > > CC:Arvid Heise
> > >
> > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > Finished
> > >
> > > > Hi, Till
> > >
> > > >
> > >
> > > > Thank you very much for your careful consideration
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > *1. Emit records in `NotifyCheckpointComplete`.*
> > >
> > > >
> > >
> > > > Sorry for making you misunderstanding because of my expression. I
> just
> > >
> > > >
> > >
> > > > want to say the current interface does not prevent users from doing
> it.
> > >
> > > >
> > >
> > > > From the perspective of the new sink api, we might not depend on
> > emitting
> > >
> > > >
> > >
> > > > records in `NotifyCheckpointComplete`, like using
> `OperatorCoordinator`
> > >
> > > >
> > >
> > > > instead.
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > *2. What does the FLIP-147 guarantee?*I think initially this FLIP
> want
> > to
> > >
> > > >
> > >
> > > > achieve two targets:
> > >
> > > >
> > >
> > > > 1. Tasks/Operators exit correctly (as you mentioned the lifecycle of
> a
> > >
> > > >
> > >
> > > > Task/StreamTask/StreamOperator.).
> > >
> > > >
> > >
> > > > 2. Continue to trigger checkpoint after some tasks for mixed jobs.
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > I think the first thing is related to the discussion in FLINK-21133.
> > If I
> > >
> > > >
> > >
> > > > understand correctly, in addition to supporting the tasks /
> operators
> > to
> > >
> > > >
> > >
> > > > exit correctly, now we also want to unify the process of the tasks
> and
> > >
> > > >
> > >
> > > > operators for savepoint / finish.
> > >
> > > >
> > >
> > > > I think the second thing is orthogonal to the FLINK-21133 because
> there
> > >
> > > are
> > >
> > > >
> > >
> > > > topologies that have both the bounded and unbounded input.
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > *3. How to unify the operator exit process of FLIP-147 with
> > >
> > > >
> > >
> > > > stop-with-savepoint?*
> > >
> > > >
> > >
> > > > I am not very sure about how to do it yet. But if I understand the
> > >
> > > >
> > >
> > > > discussion in the jira correctly it needs to introduce some logic
> into
> > >
> > > >
> > >
> > > > `CheckpointCoordinator`, which responses for triggering “the unified
> > >
> > > >
> > >
> > > > operator exit process”. Am I correct?
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > Best,
> > >
> > > >
> > >
> > > > Guowei
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann wrote:
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > > Thanks for the explanation Yun and Guowei. I have to admit that I
> do
> > >
> > > not
> > >
> > > >
> > >
> > > > > fully understand why this is strictly required but I think that we
> > are
> > >
> > > >
> > >
> > > > > touching two very important aspects which might have far fetching
> > >
> > > >
> > >
> > > > > consequences for how Flink works:
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > 1) Do we want to allow that multiple checkpoints are required to
> > >
> > > >
> > >
> > > > > materialize results?
> > >
> > > >
> > >
> > > > > 2) Do we want to allow to emit records in
> notifyCheckpointComplete?
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > For 1) I am not sure whether this has been discussed within the
> > >
> > > community
> > >
> > > >
> > >
> > > > > sufficiently. Requiring multiple checkpoints to materialize a
> result
> > >
> > > >
> > >
> > > > > because of multi level committers has the consequence that we
> > increase
> > >
> > > > the
> > >
> > > >
> > >
> > > > > latency from checkpoint interval to #levels * checkpoint interval.
> > >
> > > >
> > >
> > > > > Moreover, having to drain the pipeline in multiple steps, would
> break
> > >
> > > the
> > >
> > > >
> > >
> > > > > stop-with-savepoint --drain because which savepoint do you report
> to
> > >
> > > the
> > >
> > > >
> > >
> > > > > user?
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > For 2) allowing to send records after the final
> > >
> > > notifyCheckpointComplete
> > >
> > > >
> > >
> > > > > will effectively mean that we need to shut down a topology in
> > multiple
> > >
> > > >
> > >
> > > > > steps (in the worst case one operator per checkpoint). This would
> be
> > a
> > >
> > > >
> > >
> > > > > strong argument for not allowing this to me. The fact that users
> can
> > >
> > > send
> > >
> > > >
> > >
> > > > > records after the notifyCheckpointComplete is more by accident
> than
> > by
> > >
> > > >
> > >
> > > > > design. I think we should make this a very deliberate decision and
> in
> > >
> > > > doubt
> > >
> > > >
> > >
> > > > > I would be in favour of a more restrictive model unless there is a
> > very
> > >
> > > >
> > >
> > > > > good reason why this should be supported.
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > Taking also the discussion in FLINK-21133 [1] into account, it
> seems
> > to
> > >
> > > > me
> > >
> > > >
> > >
> > > > > that we haven't really understood what kind of guarantees we want
> to
> > >
> > > give
> > >
> > > >
> > >
> > > > > to our users and how the final checkpoint should exactly work. I
> > >
> > > > understand
> > >
> > > >
> > >
> > > > > that this is not included in the first scope of FLIP-147 but I
> think
> > >
> > > this
> > >
> > > >
> > >
> > > > > is so important that we should figure this out asap. Also because
> the
> > >
> > > > exact
> > >
> > > >
> > >
> > > > > shut down behaviour will have to be aligned with the lifecycle of
> a
> > >
> > > >
> > >
> > > > > Task/StreamTask/StreamOperator. And last but not least because
> other
> > >
> > > >
> > >
> > > > > features such as the new sink API start building upon a shut down
> > model
> > >
> > > >
> > >
> > > > > which has not been fully understood/agreed upon.
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > [1] https://issues.apache.org/jira/browse/FLINK-21133
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > Cheers,
> > >
> > > >
> > >
> > > > > Till
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma wrote:
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > > Thanks Yun for the detailed explanation.
> > >
> > > >
> > >
> > > > > > A simple supplementary explanation about the sink case: Maybe we
> > >
> > > could
> > >
> > > >
> > >
> > > > > use
> > >
> > > >
> > >
> > > > > > `OperatorCoordinator` to avoid sending the element to the
> > downstream
> > >
> > > >
> > >
> > > > > > operator.
> > >
> > > >
> > >
> > > > > > But I agree we could not limit the users not to emit records in
> the
> > >
> > > >
> > >
> > > > > > `notiyCheckpointComplete`.
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > > > Best,
> > >
> > > >
> > >
> > > > > > Guowei
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > > > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao
> > >
> > > >
> > >
> > > > > > wrote:
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > > > > Hi all,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > I'd like to first detail the issue with emitting records in
> > >
> > > >
> > >
> > > > > > > notifyCheckpointComplete for context. For specific usage,
> > >
> > > >
> > >
> > > > > > > an example would be for sink, it might want to write some
> > metadata
> > >
> > > >
> > >
> > > > > after
> > >
> > > >
> > >
> > > > > > > all the transactions are committed
> > >
> > > >
> > >
> > > > > > > (like write a marker file _SUCCESS to the output directory).
> This
> > >
> > > > case
> > >
> > > >
> > >
> > > > > is
> > >
> > > >
> > >
> > > > > > > currently supported via the two level
> > >
> > > >
> > >
> > > > > > > committers of the new sink API: when received endOfInput(),
> the
> > >
> > > >
> > >
> > > > > Committer
> > >
> > > >
> > >
> > > > > > > wait for another checkpoint to
> > >
> > > >
> > >
> > > > > > > commits all the pending transactions and emit the list of
> files
> > to
> > >
> > > > the
> > >
> > > >
> > >
> > > > > > > GlobalCommitter. The GlobalCommitter
> > >
> > > >
> > >
> > > > > > > would wait for another checkpoint to also write the metadata
> with
> > >
> > > 2pc
> > >
> > > >
> > >
> > > > > > > (Although sometimes 2pc is not needed
> > >
> > > >
> > >
> > > > > > > for writing metadata, it should be only an optimization and
> still
> > >
> > > >
> > >
> > > > > > requires
> > >
> > > >
> > >
> > > > > > > the Committer do commit before
> > >
> > > >
> > >
> > > > > > > notifying the global Committer. Also another note is
> > >
> > > GlobalCommitter
> > >
> > > > is
> > >
> > > >
> > >
> > > > > > > also added for some other cases
> > >
> > > >
> > >
> > > > > > > like some sinks want an commiter with dop = 1, like
> IceBergSink).
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > However, a more general issue to me is that currently we do
> not
> > >
> > > limit
> > >
> > > >
> > >
> > > > > > > users to not emit records in
> > >
> > > >
> > >
> > > > > > > notifyCheckpointComplete in the API level. The sink case could
> be
> > >
> > > >
> > >
> > > > > viewed
> > >
> > > >
> > >
> > > > > > > as a special case, but in addition
> > >
> > > >
> > >
> > > > > > > to this one, logically users could also implement their own
> cases
> > >
> > > > that
> > >
> > > >
> > >
> > > > > > > emits records in notifyCheckpointComplete.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > Best,
> > >
> > > >
> > >
> > > > > > > Yun
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > ------------------Original Mail ------------------
> > >
> > > >
> > >
> > > > > > > Sender:Arvid Heise
> > >
> > > >
> > >
> > > > > > > Send Date:Fri Feb 12 20:46:04 2021
> > >
> > > >
> > >
> > > > > > > Recipients:dev
> > >
> > > >
> > >
> > > > > > > CC:Yun Gao
> > >
> > > >
> > >
> > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After
> Tasks
> > >
> > > >
> > >
> > > > > Finished
> > >
> > > >
> > >
> > > > > > > Hi Piotr,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > Thank you for raising your concern. Unfortunately, I do not
> have
> > a
> > >
> > > >
> > >
> > > > > better
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > idea than doing closing of operators intermittently with
> > >
> > > checkpoints
> > >
> > > > (=
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > multiple last checkpoints).
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > However, two ideas on how to improve the overall user
> experience:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > 1. If an operator is not relying on notifyCheckpointComplete,
> we
> > >
> > > can
> > >
> > > >
> > >
> > > > > > close
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > it faster (without waiting for a checkpoint). In general, I'd
> > >
> > > assume
> > >
> > > >
> > >
> > > > > that
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > almost all non-sinks behave that way.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > 2. We may increase the checkpointing frequency for the last
> > >
> > > >
> > >
> > > > > checkpoints.
> > >
> > > >
> > >
> > > > > > We
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > need to avoid overloading checkpoint storages and task
> managers,
> > >
> > > but
> > >
> > > > I
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > assume the more operators are closed, the lower the
> checkpointing
> > >
> > > >
> > >
> > > > > > interval
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > can be.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > For 1, I'd propose to add (name TBD):
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > default boolean StreamOperator#requiresFinalCheckpoint() {
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > return true;
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > }
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > This means all operators are conservatively (=slowly) closed.
> For
> > >
> > > > most
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > operators, we can then define their behavior by overriding in
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > AbstractUdfStreamOperator
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > @Override
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > return userFunction instanceof CheckpointListener;
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > }
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > This idea can be further refined in also adding
> > >
> > > > requiresFinalCheckpoint
> > >
> > > >
> > >
> > > > > > to
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > CheckpointListener to exclude all operators with UDFs that
> > >
> > > implement
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > CheckpointListener but do not need it for 2pc.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > @Override
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > return userFunction instanceof CheckpointListener &&
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > ((CheckpointListener) userFunction).requiresFinalCheckpoint();
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > }
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > That approach would also work for statebackends/snapshot
> > strategies
> > >
> > > >
> > >
> > > > > that
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > require some 2pc.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > If we can contain it to the @PublicEvolving StreamOperator, it
> > >
> > > would
> > >
> > > > be
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > better of course.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > Best,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > Arvid
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > wrote:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Hey,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > I would like to raise a concern about implementation of the
> > final
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > checkpoints taking into account operators/functions that are
> > >
> > > >
> > >
> > > > > > implementing
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > two phase commit (2pc) protocol for exactly-once processing
> > with
> > >
> > > > some
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > external state (kept outside of the Flink). Primarily
> > >
> > > exactly-once
> > >
> > > >
> > >
> > > > > > sinks.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > First of all, as I understand it, this is not planned in the
> > >
> > > first
> > >
> > > >
> > >
> > > > > > > version
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > of this FLIP. I'm fine with that, however I would strongly
> > >
> > > > emphasize
> > >
> > > >
> > >
> > > > > > this
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > in every place we will be mentioning FLIP-147 efforts. This
> is
> > >
> > > >
> > >
> > > > > because
> > >
> > > >
> > >
> > > > > > > me,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > as a user, upon hearing "Flink supports checkpointing with
> > >
> > > bounded
> > >
> > > >
> > >
> > > > > > > inputs"
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > I would expect 2pc to work properly and to commit the
> external
> > >
> > > side
> > >
> > > >
> > >
> > > > > > > effects
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > upon finishing. As it is now, I (as a user) would be
> surprised
> > >
> > > > with a
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > silent data loss (of not committed trailing data). This is
> > just a
> > >
> > > >
> > >
> > > > > > remark,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > that we need to attach this warning to every blog
> > >
> > > >
> > >
> > > > > > post/documentation/user
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > mailing list response related to "Support Checkpoints After
> > Tasks
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Finished". Also I would suggest to prioritize the follow up
> of
> > >
> > > >
> > >
> > > > > > supporting
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 2pc.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Secondly, I think we are missing how difficult and
> problematic
> > >
> > > will
> > >
> > > >
> > >
> > > > > be
> > >
> > > >
> > >
> > > > > > > 2pc
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > support with the final checkpoint.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > For starters, keep in mind that currently 2pc can be
> > implemented
> > >
> > > by
> > >
> > > >
> > >
> > > > > > users
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > using both `@Public` APIs as functions and `@PublicEvolving`
> > >
> > > >
> > >
> > > > > operators
> > >
> > > >
> > >
> > > > > > in
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > any place in the job graph. It's not limited to only the
> sinks.
> > >
> > > For
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > example users could easily implement the `AsynFunction` (for
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `AsyncWaitOperator`) that is using 2pc based on the
> > >
> > > >
> > >
> > > > > > `CheckpointListener`
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > interface. I'm not saying it's common, probably just a tiny
> > >
> > > > minority
> > >
> > > >
> > >
> > > > > of
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > users are doing that (if any at all), but nevertheless
> that's
> > >
> > > >
> > >
> > > > > possible
> > >
> > > >
> > >
> > > > > > > and
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > currently (implicitly?) supported in Flink.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Next complication is the support of bounded streams
> > >
> > > >
> > >
> > > > > (`BoundedOneInput`
> > >
> > > >
> > >
> > > > > > or
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `BoundedMultiInput` interfaces) and the closing/shutdown
> > >
> > > procedure
> > >
> > > > of
> > >
> > > >
> > >
> > > > > > the
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > operators. Currently it works as follows:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 1. `endOfInput` is called on the first operator in the chain
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 2. We quiesce the processing timers
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> > for
> > >
> > > > the
> > >
> > > >
> > >
> > > > > > > first
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > operator, so no new timers will be triggered
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 3. We wait for the already fired timers to finish executing
> > >
> > > > (spinning
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > mailbox loop)
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 4. We are closing the first operator
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 5. We go to the next (second) operator in the chain and
> repeat
> > >
> > > the
> > >
> > > >
> > >
> > > > > > steps
> > >
> > > >
> > >
> > > > > > > 1.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > to 5.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > This is because operators can emit data after processing
> > >
> > > >
> > >
> > > > > `endOfInput`,
> > >
> > > >
> > >
> > > > > > > from
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > timers, async mailbox actions and inside the `close` method
> > >
> > > itself.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Now the problem is to support the final checkpoint with 2pc,
> we
> > >
> > > > need
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > trigger `snapshotState` and `notifyCheckpointComplete` call
> at
> > >
> > > the
> > >
> > > >
> > >
> > > > > very
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > least only after `endOfInput` call on the operator. Probably
> > the
> > >
> > > > best
> > >
> > > >
> > >
> > > > > > > place
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > would be in between steps 3. and 4. However that means, we
> > would
> > >
> > > be
> > >
> > > >
> > >
> > > > > > > forced
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > to wait for steps 1. to 3. to finish, then wait for a next
> > >
> > > > checkpoint
> > >
> > > >
> > >
> > > > > > to
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > trigger AND complete, before finally closing the head
> operator,
> > >
> > > and
> > >
> > > >
> > >
> > > > > > only
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > then we can start closing the next operator in the chain:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 1. `endOfInput` is called on the first operator in the chain
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 2. We quiesce the processing timers
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> > for
> > >
> > > > the
> > >
> > > >
> > >
> > > > > > > first
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > operator, so no new timers will be triggered
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 3. We wait for the already fired timers to finish executing
> > >
> > > > (spinning
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > mailbox loop)
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > *3b. We wait for one more checkpoint to trigger and for the
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `notifyCheckpointComplete` RPC.*
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 4. We are closing the first operator
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 5. We go to the next (second) operator in the chain and
> repeat
> > >
> > > the
> > >
> > > >
> > >
> > > > > > steps
> > >
> > > >
> > >
> > > > > > > 1.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > to 5.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > That means, we can close one operator per successful
> > checkpoint.
> > >
> > > To
> > >
> > > >
> > >
> > > > > > close
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 10 operators, we would need 10 successful checkpoints.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > I was thinking about different approaches to this problem,
> and
> > I
> > >
> > > >
> > >
> > > > > > couldn't
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > find any viable ones. All I could think of would break the
> > >
> > > current
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `@Public` API and/or would be ugly/confusing for the users.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > For example a relatively simple solution, to introduce a
> > >
> > > `preClose`
> > >
> > > >
> > >
> > > > > or
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `flush` method to the operators, with a contract that after
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `flush`, operators would be forbidden from emitting more
> > records,
> > >
> > > > so
> > >
> > > >
> > >
> > > > > > that
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > we can replace step 4. with this `flush` call, and then
> having
> > a
> > >
> > > >
> > >
> > > > > single
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > checkpoint to finish 2pc for all of the operators inside the
> > >
> > > chain,
> > >
> > > >
> > >
> > > > > > > doesn't
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > work. Sheer fact of adding this `flush` method and changing
> the
> > >
> > > >
> > >
> > > > > > contract
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > would break the current API and Yun Gao has pointed out to
> me,
> > >
> > > that
> > >
> > > >
> > >
> > > > > we
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > either already support, or want to support operators that
> are
> > >
> > > >
> > >
> > > > > emitting
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > records from within the `notifyCheckpointComplete` call:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Yun Gao:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > like with the new sink api there might be writer ->
> committer
> > >
> > > ->
> > >
> > > >
> > >
> > > > > > global
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > committer, the committer would need to wait for the last
> > >
> > > checkpoint
> > >
> > > >
> > >
> > > > > to
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > commit
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > the last piece of data, and after that it also need to
> emit
> > the
> > >
> > > >
> > >
> > > > > list
> > >
> > > >
> > >
> > > > > > of
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > transactions get committed to global committer to do some
> > >
> > > >
> > >
> > > > > finalization
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > logic.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > So it wouldn't solve the problem (at least not fully).
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > I don't know if anyone has any better ideas how to solve
> this
> > >
> > > >
> > >
> > > > > problem?
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Piotrek
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > pt., 15 sty 2021 o 14:57 Yun Gao
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > napisał(a):
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Hi Aljoscha,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > I think so since we seems to do not have other divergence
> and
> > >
> > > new
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > objections now. I'll open the vote then. Very thanks!
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Best,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Yun
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > > ------------------------------------------------------------------
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > From:Aljoscha Krettek
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Send Time:2021 Jan. 15 (Fri.) 21:24
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > To:dev
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After
> > Tasks
> > >
> > > >
> > >
> > > > > > > Finished
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Thanks for the summary! I think we can now move towards a
> > >
> > > [VOTE]
> > >
> > > >
> > >
> > > > > > > thread,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > right?
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > On 2021/01/15 13:43, Yun Gao wrote:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >1) For the problem that the "new" root task coincidently
> > >
> > > > finished
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >before getting triggered successfully, we have listed two
> > >
> > > > options
> > >
> > > >
> > >
> > > > > in
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >the FLIP-147[1], for the first version, now we are not
> tend
> > to
> > >
> > > > go
> > >
> > > >
> > >
> > > > > > with
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >the first option that JM would re-compute and re-trigger
> new
> > >
> > > >
> > >
> > > > > sources
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >when it realized some tasks are not triggered
> successfully.
> > >
> > > This
> > >
> > > >
> > >
> > > > > > > option
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >would avoid the complexity of adding new PRC and
> duplicating
> > >
> > > > task
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >states, and in average case it would not cause too much
> > >
> > > > overhead.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > You wrote "we are *not* tend to go with the first option",
> > but
> > >
> > > I
> > >
> > > >
> > >
> > > > > > think
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > you meant wo write "we tend to *now* go with the first
> > option",
> > >
> > > >
> > >
> > > > > > right?
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > That's also how it is in the FLIP, I just wanted to
> clarify
> > for
> > >
> > > > the
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > mailing list.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > >
> > >
> > >
> >
>
>


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

Posted by Kezhu Wang <ke...@gmail.com>.
In “stop-with-savepoint —drain”, MAX_WATERMARK is not an issue. For normal
finishing task, not allowing unaligned checkpoint does not solve the
problem as MAX_WATERMARK could be persisted in downstream task. When
scenario @Piotr depicted occurs, downstream(or further downstream) window
operator will count all inputs as late.

> If we ensure that the MAX_WATERMARK is only persisted in state if a
recovery will trigger immediately the shut down of this operator, then it
shouldn't be an issue.

You are right in case the assumption holds, I have same thought as you
before. But I am kind of worry about whether it is too prefect to be
fragile. This requires strong guarantee from implementation that a recovery
from TERMINATING stage should go directly to that stage.

> I think the MAX_WATERMARK event should be sent either just before or with
the EndOfPartitionEvent.

I prefer “with the EndOfPartitionEvent”. EndOfPartitionEvent itself already
carry what ending MAX_WATERMARK try to express. May be we can reuse it ?
@Piotr

A preview work[1] from @Yun in PR#14831 explains EndOfPartitionEvent as
checkpoint barrier if there are pending checkpoints.


[1]:
https://github.com/gaoyunhaii/flink/commit/1aaa80fb0afad13a314b06783c61c01b9414f91b#diff-d4568b34060bc589cd1354bd125c35aca993dd0c9fe9e4460dfed73501116459R177


Best,
Kezhu Wang

On February 28, 2021 at 21:23:31, Till Rohrmann (trohrmann@apache.org)
wrote:

I think you are right with the problem of endOfInput. endOfInput should not
be used to commit final results. In fact if this termination fails then we
might end up in a different outcome of the job which is equally valid as
the one before the failure.

Concerning unaligned checkpoints, I think they don't play well together
with draining a streaming pipeline. The problem is that in the draining
case you want to process all records which are still in flight but
unaligned checkpoints don't guarantee this as they can jump in flight
records.

I think the MAX_WATERMARK event should be sent either just before or with
the EndOfPartitionEvent. If we ensure that the MAX_WATERMARK is only
persisted in state if a recovery will trigger immediately the shut down of
this operator, then it shouldn't be an issue.

Cheers,
Till

On Sun, Feb 28, 2021 at 9:06 AM Kezhu Wang <ke...@gmail.com> wrote:

> Hi Till,
>
> Just for bookkeeping, some observations from current implementation.
>
> > With this model, the final checkpoint is quite simple because it is
> ingrained in the lifecycle of an operator. Differently said an operator
> will only terminate after it has committed its side effects and seen the
> notifyCheckpointComplete message (if it is stateful).
>
> Currently, we could not mark this operator(or subtask) as terminated since
> result of `notifyCheckpointComplete`(possible side effect committing) is
> not taken into account of the belonging checkpoint. The job has to run to
> next safe point(finished or next checkpoint success) to be marked as
> “terminated”.
>
> > How to implement the stop-with-savepoint --drain/terminate command with
> this model: One idea could be to tell the sources that they should stop
> reading. This should trigger the EndOfPartitionEvent to be sent
> downstream.
> This will transition all operators into the TERMINATING state.
>
> Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
> returned. To achieve above, possible works should be required:
> * Promote  `EndOfPartitionEvent` from `Task` to `StreamTask`. This may
> have some interferences with BatchTask or network io stack.
> * Or introducing stream task level `EndOfUserRecordsEvent`(from PR#14831
> @Yun @Piotr)
> * Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.
>
> Besides this, I would like to quote some discussion from FLINK-21467
> between @Piotr and me:
>
> From @Piotr
> > Note, that it's not only that endOfInput can be called multiple times.
> There is a very remote possibility that the following scenario will happen:
> 1. checkpoint is taken (successfully)
> 2. sources are finishing
> 3. endOfInput is issued
> 4. job fails
> 5. job restarts to checkpoint 1.
> 6. after failover, because of some non deterministic logic in the source,
> sources are not finishing
>
> From me
> > But I think there is little work Flink can do to cope with this kind of
> issues. The checkpoint could be a savepoint triggered from user side and
> the "non deterministic logic" could be a change from user(eg. changing of
> stoppingOffsets in KafkaSource).
>
> > I think the "non deterministic logic" could cause trouble in combination
> with unaligned checkpoint and downstream window operator. Unaligned
> checkpoint will persist "MAX_WATERMARK" in state, after restarting,
> "MAX_WATERMARK" will cause downstream window operator ignores all future
> inputs.
>
> FLIP-147 demands no new records from end-of-stream-flushing, but source
> will emit “MAX_WATERMARK” on ending. Previously, I thought it is not a
> valid issue, but turn out that it could cause trouble under scenario listed
> by @Piotr if I am not wrong.
>
>
> PR#14831: https://github.com/apache/flink/pull/14831
> FLINK-21467: https://issues.apache.org/jira/browse/FLINK-21467
>
>
> Best,
> Kezhu Wang
>
> On February 27, 2021 at 18:12:20, Till Rohrmann (trohrmann@apache.org)
> wrote:
>
> Thanks for all your thoughts. I think we should further think through
> whether to allow checkpoints after an operator has emitted all its records
> (e.g. after close is called currently) or not. I think by doing this we
> would nicely decouple the checkpoint taking from the operator lifecycle
> and
> wouldn't need special checkpoints/savepoints for the final checkpoint and
> stop-with-savepoint --drain. Let me try to explain this a bit more
> detailed.
>
> If we say an operator has the rough lifecycle RUNNING => TERMINATING =>
> TERMINATED where we go from RUNNING into TERMINATING after we have seen
> the
> EndOfPartitionEvent and flushed all our records. The operator goes from
> TERMINATING => TERMINATED if it has persisted all its possible side
> effects. Throughout all states, it is possible to trigger a checkpoint. A
> stateless operator will immediately go from TERMINATING to TERMINATED
> whereas a stateful operator would wait for another checkpoint to be
> triggered and successfully completed (notifyCheckpointComplete).
>
> With this model, the final checkpoint is quite simple because it is
> ingrained in the lifecycle of an operator. Differently said an operator
> will only terminate after it has committed its side effects and seen the
> notifyCheckpointComplete message (if it is stateful). Here it is important
> to note that in the streaming case, different bounded operators can
> terminate at different times. They don't have to terminate all with the
> same checkpoint.
>
> How to implement the stop-with-savepoint --drain/terminate command with
> this model: One idea could be to tell the sources that they should stop
> reading. This should trigger the EndOfPartitionEvent to be sent
> downstream.
> This will transition all operators into the TERMINATING state. Next the JM
> can trigger a checkpoint to shut the operators down and not to wait for
> the
> next automatic checkpoint trigger event.
>
> By allowing checkpoints throughout the entire lifecycle of an operator we
> disallow sending records from notifyCheckpointComplete because this
> message
> will also be sent in the state TERMINATING where an operator has already
> produced all of its records.
>
> What do you think? Maybe this model overlooks some important cases. One
> downside is that we will break the operator API with changing the
> lifecycle
> of an operator.
>
> Cheers,
> Till
>
>
>
> On Fri, Feb 26, 2021 at 7:29 AM Kezhu Wang <ke...@gmail.com> wrote:
>
> > Hi Yun,
> >
> > The termination phase I drafted depends on a de-fact that
> > `notifyCheckpointComplete` is ignored currently after `close`. Thus, any
> > external materialization inside `close`(including `endInput`) or
> > `notifyCheckpointComplete` is either not guaranteed or not committed.
> >
> > I tried to emphasize no-breaking changes and post-pone migration in
> later
> > releases for this termination phase. But now, I am kind of worry about
> > whether it will cause long-term maintenance hell.
> >
> > Personally, I think allowing checkpoint after `close`(@Till proposed
> this
> > in FLINK-21133) could be the minimal change. But there are concerns:
> > * It will break some existing code possibly in a silent way.(@Piotr
> pointed
> > this already in FLINK-21133)
> > * I think the name `close` is kind of misleading. (@Piotr suggested
> > renaming this to `finish` to not break code silently in FLINK-21133)
> >
> > > stop-with-savepoint --drain would wait for a specific savepoint
> >
> > For stop-with-savepoint, the checkpoint barrier is already created or
> > received there.
> >
> >
> > Best,
> > Kezhu Wang
> >
> > On February 25, 2021 at 21:02:30, Yun Gao (yungao.gy@aliyun.com) wrote:
> >
> > Hi all,
> >
> > Very thanks for the discussions!
> >
> >
> >
> > A. Regarding how to avoid emitting records in notifyCheckpointComplete:
> >
> > Currently the structure of a new sink is writer -> committer -> global
> > committer and the paralellism of global committer
> > must be one. By design it would be used in several cases:
> > 1. writer -> committer: for normal sinks that write pending data in
> writer
> > and commit pending data in committer.
> > 2. writer -> global committer: for sinks require the committer's
> > parallelism be 1, like iceberg sink which
> > relies on optimistic lock to commit data hopes to reduce the conflicts.
> > 3. writer -> committer -> global committer: for sinks that also need to
> > write meta data (like _SUCCESS file
> > or add record in hive metastore) in global committer.
> >
> > The third case would cause the cascade commit problem. To overcome this
> > problem, we would like to
> > 1. Previously the global committer also support writing metadata with
> 2pc.
> > Now we disable this mode
> > and always rely on the property that writing metadata is repeatable.
> With
> > this limitation there should no
> > need of cascade commit, but the committer would still emit records in
> > notifyCheckpointComplete.
> > 2. We further move global committer in the case 3 to the operator
> > coordinator. Then the committer would
> > not need to emit records, but communicates with its operator
> coordinator.
> >
> > One core issue for using the OperatorCoordinator is how to keep the
> > communication between the operator
> > and the OperatorCoordinator exactly-once. Since the message is always
> from
> > the oeprator to the OperatorCoordinator
> > in this case, we would only need to bookkeep the message sent between
> the
> > OperatorCoordinator takes snapshot and
> > the Operator takes snapshot in the state of the Operator.
> >
> > On how to achieve the change in detail we would still need some think,
> it
> > currently seems we would have to had
> > some modification to the current new sink api.
> >
> >
> >
> > B. Regarding the stop-with-savepoint --drain
> >
> > Very thanks @Piotr for the further explanation and now I realize I have
> > understand wrongly for the semantics of
> > stop-with-savepoint --drain. Now I think that the problem should be we
> > should also include the records produced in
> > `endOfInput()` and `close()` also in the last savepoint, am I correct?
> If
> > so, it seems we still have some undetermined options for
> > the lifecycle of the operator, like in Kezhu's proposal the close()
> happens
> > at last, but it seems close() might also emit records (
> > so now the operator are closed with op1's close() -> op2's endOfInput()
> ->
> > op2's close() -> op3's endOfinput -> ...) ?
> >
> > And on the other side, as Kezhu has also proposed, perhapse we might
> have
> > the stop-with-savepoint --drain and normal exit in the same process,
> > but have slightly difference in that stop-with-savepoint --drain would
> wait
> > for a specific savepoint and in normal exit, the operator
> > might wait for arbitrary checkpoint. If we could achieve not emitting
> > records in notifyCheckpointComplete, stop-with-savepoint --drain could
> > be done with one savepoint, and for the normal exit, the operator would
> not
> > need to wait for other slow operators to exit.
> >
> > Best,
> > Yun
> >
> >
> >
> > ------------------Original Mail ------------------
> > *Sender:*Kezhu Wang <ke...@gmail.com>
> > *Send Date:*Thu Feb 25 15:11:53 2021
> > *Recipients:*Flink Dev <de...@flink.apache.org>, Piotr Nowojski <
> > piotr.nowojski@gmail.com>
> > *CC:*Guowei Ma <gu...@gmail.com>, jingsonglee0@gmail.com <
> > jingsonglee0@gmail.com>
> > *Subject:*Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > Finished
> >
> > > Hi all, thanks for driving this and especially Piotr for re-active
> this
> > >
> > > thread.
> > >
> > >
> > >
> > > First, for `notifyCheckpointComplete`, I have strong preference
> towards
> > >
> > > "shut down the dataflow
> > >
> > > pipeline with one checkpoint in total", so I tend to option dropping
> > "send
> > >
> > > records" from
> > >
> > > `notifyCheckpointComplete` for next level committing in pipeline, if
> we
> > >
> > > ever support it. Without this,
> > >
> > > we are unable to stop a pipeline manually with all results
> materialized.
> > >
> > >
> > >
> > > Second, for shutdown unification of `stop-with-savepoint --drain` and
> > >
> > > FLIP-147, I draft following
> > >
> > > phase based on emerging proposals(mainly by Piotr and Till) in this
> > thread
> > >
> > > and FLINK-21133.
> > >
> > >
> > >
> > > ```java
> > >
> > > // StreamTask termination phase
> > >
> > > finish()(Call StreamOperator.finish in chaining order)
> > >
> > > advanceToEndOfEventTime()(nop if for no-source inputs, this could also
> be
> > >
> > > done in input processor or finish ?)
> > >
> > > if (there-is-a-pending-terminate-savepoint) {
> > >
> > > triggerBarrierForDownStream();
> > >
> > > waitCheckpointComplete();
> > >
> > > } else if (require-two-phase-commit-in-shutdown) {
> > >
> > > waitFinalCheckpoint();
> > >
> > > waitCheckpointComplete();
> > >
> > > }
> > >
> > > // TODO: Ignore/Cancel all checkpoints from this point(or hit to
> > >
> > > coordinator in final checkpoint response ?).
> > >
> > > close();
> > >
> > > ```
> > >
> > >
> > >
> > > Branches in `if` could reside in different code paths, but the effect
> > >
> > > should be similar to above.
> > >
> > > The divergence reflects fact that "stop-with-savepoint --drain" is
> > >
> > > triggered already while we need
> > >
> > > to wait for final checkpoint in natural input exhausting.
> > >
> > >
> > >
> > > With carefully chosen default functions, we will not break existing
> > >
> > > interfaces.
> > >
> > > * `StreamOperator.finish`
> > >
> > > * `RichFunction.finish`
> > >
> > >
> > >
> > > For existing codes:
> > >
> > > 1. "stop-with-savepoint" with no migration, it should behaves as
> before.
> > >
> > > 2. "require-two-phase-commit-in-shutdown" evaluates to false and no
> > >
> > > migration, it is same as before.
> > >
> > > 3. "require-two-phase-commit-in-shutdown" evaluates to true and no
> > >
> > > migration, records in `close` will
> > >
> > > lose.
> > >
> > >
> > >
> > > For case#3:
> > >
> > > A. No external materialization in `StreamOperator.close`: Nothing to
> > lose.
> > >
> > > B. External materialization only in `StreamOperator.close`: I don't
> think
> > >
> > > Flink ever claimed that there is
> > >
> > > any guarantee for this situation.
> > >
> > > C. External materialization in `notifyCheckpointComplete` from
> > >
> > > `StreamOperator.close`: But due to fact that
> > >
> > > `notifyCheckpointComplete` was ignored after operator
> > >
> > > closed(FLINK-16383), so there will be no external
> > >
> > > materialization to lose.
> > >
> > >
> > >
> > > Then, we could recommend users to migrate possible
> > “end-of-stream-flushing”
> > >
> > > from “close" to “finish”.
> > >
> > >
> > >
> > > Best,
> > >
> > > Kezhu Wang
> > >
> > >
> > >
> > >
> > >
> > > On February 24, 2021 at 23:47:36, Piotr Nowojski (
> > piotr.nowojski@gmail.com
> > > )
> > >
> > > wrote:
> > >
> > >
> > >
> > > Thanks for the reponses Guowei and Yun,
> > >
> > >
> > >
> > > Could you elaborate more/remind me, what does it mean to replace
> emitting
> > >
> > > results from the `notifyCheckpointComplete` with `OperatorCoordinator`
> > >
> > > approach?
> > >
> > >
> > >
> > > About the discussion in FLINK-21133 and how it relates to FLIP-147.
> You
> > are
> > >
> > > right Yun gao, that in case of `stop-with-savepoint --drain` the whole
> > job
> > >
> > > finishes, while in FLIP-147 some of the sources can work for
> arbitrarily
> > >
> > > long time after others have already finished. However from the runtime
> > >
> > > perspective, especially how to close a Task and it's operators, there
> is
> > no
> > >
> > > difference between those two cases. In both cases, we need to end
> input,
> > >
> > > shut down processing time timers and commit external side effects (two
> > >
> > > phase commit support) before exiting the task. That's how the
> discussion
> > >
> > > about the behaviour of "stop-with-savepoint" was connected with
> FLIP-147.
> > >
> > >
> > >
> > > Currently on master, "stop-with-savepoint --drain" drains/flushes
> > buffered
> > >
> > > records and deals correctly with timers, but all of that happens AFTER
> > >
> > > savepoint was completed. So any records flushed from the operators
> during
> > >
> > > endOfInput/close/shutting down processing timers are never committed
> to
> > >
> > > external systems. This is exactly the same problem as the "two phase
> > >
> > > commit" problem of FLIP-147, that should have the same solution and it
> > >
> > > should be solved at the same time. For example if we go with the *very
> > >
> > > undesirable* "one closed operator per one completed
> > checkpoint/savepoint",
> > >
> > > in both cases CheckpointCoordinator, Scheduler and Task would need to
> > keep
> > >
> > > the task alive and keep triggering checkpoints for that task, until
> all
> > >
> > > operators in the operator chain are closed (one closed operator per
> one
> > >
> > > completed checkpoint).
> > >
> > >
> > >
> > > Piotrek
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > > śr., 24 lut 2021 o 11:30 Yun Gao napisał(a):
> > >
> > >
> > >
> > > > Hi Till, Guowei,
> > >
> > > >
> > >
> > > > Very thanks for initiating the disucssion and the deep thoughts!
> > >
> > > >
> > >
> > > > For the notifyCheckpointComplete, I also agree we could try to avoid
> > >
> > > > emitting new records in notifyCheckpointComplete via using
> > >
> > > > OperatorCoordinator
> > >
> > > > for new sink API. Besides, the hive sink might also need some
> > >
> > > modification
> > >
> > > > for it also emits records in notifyCheckpointComplete.
> > >
> > > >
> > >
> > > > For unifying the process of stopping with savepoint and finished due
> to
> > >
> > > > all records
> > >
> > > > are processed, I also agree with that unifying would always be
> better
> > if
> > >
> > > > we could achieve,
> > >
> > > > but I'm still not fully catch up with the implementation: Based on
> the
> > >
> > > > discussion in FLINK-21133,
> > >
> > > > my understanding is that for stopping with savepoint, now we want to
> > >
> > > first
> > >
> > > > stop the source, then we
> > >
> > > > trigger a savepoint, and after the source received
> > >
> > > > notifyCheckpointComplete, the source would
> > >
> > > > start emitting EndOfPartitionEvent to finish the job, am I correct ?
> > >
> > > >
> > >
> > > > For normal finish, a difference to me might be if we have multiple
> > >
> > > > sources, we could not guarantee
> > >
> > > > when the sources are to finish. We might have one source run one 1
> > minute
> > >
> > > > and another one run for
> > >
> > > > 1 hour. To unify with the process with stop with savepoint, we might
> > need
> > >
> > > > to hold the fast source until
> > >
> > > > all the sources are finished? An coordinator would be introduced to
> > count
> > >
> > > > the number of sources
> > >
> > > > runing and trigger the final savepoint / checkpoint. For the extreme
> > >
> > > > cases, if we have both bounded and
> > >
> > > > unbounded sources, we might only count how much bounded source are
> > >
> > > > remaining ? And if all the bounded
> > >
> > > > sources are finished we would trigger the special checkpoint. After
> all
> > >
> > > > the bounded part of the graph are
> > >
> > > > finished, the the remaining part could still do checkpoint and
> commit
> > >
> > > data
> > >
> > > > with FLIP-147.
> > >
> > > >
> > >
> > > > Best,
> > >
> > > > Yun
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > ------------------Original Mail ------------------
> > >
> > > > Sender:Guowei Ma
> > >
> > > > Send Date:Wed Feb 24 17:35:36 2021
> > >
> > > > Recipients:dev
> > >
> > > > CC:Arvid Heise
> > >
> > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > Finished
> > >
> > > > Hi, Till
> > >
> > > >
> > >
> > > > Thank you very much for your careful consideration
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > *1. Emit records in `NotifyCheckpointComplete`.*
> > >
> > > >
> > >
> > > > Sorry for making you misunderstanding because of my expression. I
> just
> > >
> > > >
> > >
> > > > want to say the current interface does not prevent users from doing
> it.
> > >
> > > >
> > >
> > > > From the perspective of the new sink api, we might not depend on
> > emitting
> > >
> > > >
> > >
> > > > records in `NotifyCheckpointComplete`, like using
> `OperatorCoordinator`
> > >
> > > >
> > >
> > > > instead.
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > *2. What does the FLIP-147 guarantee?*I think initially this FLIP
> want
> > to
> > >
> > > >
> > >
> > > > achieve two targets:
> > >
> > > >
> > >
> > > > 1. Tasks/Operators exit correctly (as you mentioned the lifecycle of
> a
> > >
> > > >
> > >
> > > > Task/StreamTask/StreamOperator.).
> > >
> > > >
> > >
> > > > 2. Continue to trigger checkpoint after some tasks for mixed jobs.
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > I think the first thing is related to the discussion in FLINK-21133.
> > If I
> > >
> > > >
> > >
> > > > understand correctly, in addition to supporting the tasks /
> operators
> > to
> > >
> > > >
> > >
> > > > exit correctly, now we also want to unify the process of the tasks
> and
> > >
> > > >
> > >
> > > > operators for savepoint / finish.
> > >
> > > >
> > >
> > > > I think the second thing is orthogonal to the FLINK-21133 because
> there
> > >
> > > are
> > >
> > > >
> > >
> > > > topologies that have both the bounded and unbounded input.
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > *3. How to unify the operator exit process of FLIP-147 with
> > >
> > > >
> > >
> > > > stop-with-savepoint?*
> > >
> > > >
> > >
> > > > I am not very sure about how to do it yet. But if I understand the
> > >
> > > >
> > >
> > > > discussion in the jira correctly it needs to introduce some logic
> into
> > >
> > > >
> > >
> > > > `CheckpointCoordinator`, which responses for triggering “the unified
> > >
> > > >
> > >
> > > > operator exit process”. Am I correct?
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > Best,
> > >
> > > >
> > >
> > > > Guowei
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann wrote:
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > > Thanks for the explanation Yun and Guowei. I have to admit that I
> do
> > >
> > > not
> > >
> > > >
> > >
> > > > > fully understand why this is strictly required but I think that we
> > are
> > >
> > > >
> > >
> > > > > touching two very important aspects which might have far fetching
> > >
> > > >
> > >
> > > > > consequences for how Flink works:
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > 1) Do we want to allow that multiple checkpoints are required to
> > >
> > > >
> > >
> > > > > materialize results?
> > >
> > > >
> > >
> > > > > 2) Do we want to allow to emit records in
> notifyCheckpointComplete?
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > For 1) I am not sure whether this has been discussed within the
> > >
> > > community
> > >
> > > >
> > >
> > > > > sufficiently. Requiring multiple checkpoints to materialize a
> result
> > >
> > > >
> > >
> > > > > because of multi level committers has the consequence that we
> > increase
> > >
> > > > the
> > >
> > > >
> > >
> > > > > latency from checkpoint interval to #levels * checkpoint interval.
> > >
> > > >
> > >
> > > > > Moreover, having to drain the pipeline in multiple steps, would
> break
> > >
> > > the
> > >
> > > >
> > >
> > > > > stop-with-savepoint --drain because which savepoint do you report
> to
> > >
> > > the
> > >
> > > >
> > >
> > > > > user?
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > For 2) allowing to send records after the final
> > >
> > > notifyCheckpointComplete
> > >
> > > >
> > >
> > > > > will effectively mean that we need to shut down a topology in
> > multiple
> > >
> > > >
> > >
> > > > > steps (in the worst case one operator per checkpoint). This would
> be
> > a
> > >
> > > >
> > >
> > > > > strong argument for not allowing this to me. The fact that users
> can
> > >
> > > send
> > >
> > > >
> > >
> > > > > records after the notifyCheckpointComplete is more by accident
> than
> > by
> > >
> > > >
> > >
> > > > > design. I think we should make this a very deliberate decision and
> in
> > >
> > > > doubt
> > >
> > > >
> > >
> > > > > I would be in favour of a more restrictive model unless there is a
> > very
> > >
> > > >
> > >
> > > > > good reason why this should be supported.
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > Taking also the discussion in FLINK-21133 [1] into account, it
> seems
> > to
> > >
> > > > me
> > >
> > > >
> > >
> > > > > that we haven't really understood what kind of guarantees we want
> to
> > >
> > > give
> > >
> > > >
> > >
> > > > > to our users and how the final checkpoint should exactly work. I
> > >
> > > > understand
> > >
> > > >
> > >
> > > > > that this is not included in the first scope of FLIP-147 but I
> think
> > >
> > > this
> > >
> > > >
> > >
> > > > > is so important that we should figure this out asap. Also because
> the
> > >
> > > > exact
> > >
> > > >
> > >
> > > > > shut down behaviour will have to be aligned with the lifecycle of
> a
> > >
> > > >
> > >
> > > > > Task/StreamTask/StreamOperator. And last but not least because
> other
> > >
> > > >
> > >
> > > > > features such as the new sink API start building upon a shut down
> > model
> > >
> > > >
> > >
> > > > > which has not been fully understood/agreed upon.
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > [1] https://issues.apache.org/jira/browse/FLINK-21133
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > Cheers,
> > >
> > > >
> > >
> > > > > Till
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma wrote:
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > > Thanks Yun for the detailed explanation.
> > >
> > > >
> > >
> > > > > > A simple supplementary explanation about the sink case: Maybe we
> > >
> > > could
> > >
> > > >
> > >
> > > > > use
> > >
> > > >
> > >
> > > > > > `OperatorCoordinator` to avoid sending the element to the
> > downstream
> > >
> > > >
> > >
> > > > > > operator.
> > >
> > > >
> > >
> > > > > > But I agree we could not limit the users not to emit records in
> the
> > >
> > > >
> > >
> > > > > > `notiyCheckpointComplete`.
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > > > Best,
> > >
> > > >
> > >
> > > > > > Guowei
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > > > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao
> > >
> > > >
> > >
> > > > > > wrote:
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > > > > Hi all,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > I'd like to first detail the issue with emitting records in
> > >
> > > >
> > >
> > > > > > > notifyCheckpointComplete for context. For specific usage,
> > >
> > > >
> > >
> > > > > > > an example would be for sink, it might want to write some
> > metadata
> > >
> > > >
> > >
> > > > > after
> > >
> > > >
> > >
> > > > > > > all the transactions are committed
> > >
> > > >
> > >
> > > > > > > (like write a marker file _SUCCESS to the output directory).
> This
> > >
> > > > case
> > >
> > > >
> > >
> > > > > is
> > >
> > > >
> > >
> > > > > > > currently supported via the two level
> > >
> > > >
> > >
> > > > > > > committers of the new sink API: when received endOfInput(),
> the
> > >
> > > >
> > >
> > > > > Committer
> > >
> > > >
> > >
> > > > > > > wait for another checkpoint to
> > >
> > > >
> > >
> > > > > > > commits all the pending transactions and emit the list of
> files
> > to
> > >
> > > > the
> > >
> > > >
> > >
> > > > > > > GlobalCommitter. The GlobalCommitter
> > >
> > > >
> > >
> > > > > > > would wait for another checkpoint to also write the metadata
> with
> > >
> > > 2pc
> > >
> > > >
> > >
> > > > > > > (Although sometimes 2pc is not needed
> > >
> > > >
> > >
> > > > > > > for writing metadata, it should be only an optimization and
> still
> > >
> > > >
> > >
> > > > > > requires
> > >
> > > >
> > >
> > > > > > > the Committer do commit before
> > >
> > > >
> > >
> > > > > > > notifying the global Committer. Also another note is
> > >
> > > GlobalCommitter
> > >
> > > > is
> > >
> > > >
> > >
> > > > > > > also added for some other cases
> > >
> > > >
> > >
> > > > > > > like some sinks want an commiter with dop = 1, like
> IceBergSink).
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > However, a more general issue to me is that currently we do
> not
> > >
> > > limit
> > >
> > > >
> > >
> > > > > > > users to not emit records in
> > >
> > > >
> > >
> > > > > > > notifyCheckpointComplete in the API level. The sink case could
> be
> > >
> > > >
> > >
> > > > > viewed
> > >
> > > >
> > >
> > > > > > > as a special case, but in addition
> > >
> > > >
> > >
> > > > > > > to this one, logically users could also implement their own
> cases
> > >
> > > > that
> > >
> > > >
> > >
> > > > > > > emits records in notifyCheckpointComplete.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > Best,
> > >
> > > >
> > >
> > > > > > > Yun
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > ------------------Original Mail ------------------
> > >
> > > >
> > >
> > > > > > > Sender:Arvid Heise
> > >
> > > >
> > >
> > > > > > > Send Date:Fri Feb 12 20:46:04 2021
> > >
> > > >
> > >
> > > > > > > Recipients:dev
> > >
> > > >
> > >
> > > > > > > CC:Yun Gao
> > >
> > > >
> > >
> > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After
> Tasks
> > >
> > > >
> > >
> > > > > Finished
> > >
> > > >
> > >
> > > > > > > Hi Piotr,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > Thank you for raising your concern. Unfortunately, I do not
> have
> > a
> > >
> > > >
> > >
> > > > > better
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > idea than doing closing of operators intermittently with
> > >
> > > checkpoints
> > >
> > > > (=
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > multiple last checkpoints).
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > However, two ideas on how to improve the overall user
> experience:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > 1. If an operator is not relying on notifyCheckpointComplete,
> we
> > >
> > > can
> > >
> > > >
> > >
> > > > > > close
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > it faster (without waiting for a checkpoint). In general, I'd
> > >
> > > assume
> > >
> > > >
> > >
> > > > > that
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > almost all non-sinks behave that way.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > 2. We may increase the checkpointing frequency for the last
> > >
> > > >
> > >
> > > > > checkpoints.
> > >
> > > >
> > >
> > > > > > We
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > need to avoid overloading checkpoint storages and task
> managers,
> > >
> > > but
> > >
> > > > I
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > assume the more operators are closed, the lower the
> checkpointing
> > >
> > > >
> > >
> > > > > > interval
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > can be.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > For 1, I'd propose to add (name TBD):
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > default boolean StreamOperator#requiresFinalCheckpoint() {
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > return true;
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > }
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > This means all operators are conservatively (=slowly) closed.
> For
> > >
> > > > most
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > operators, we can then define their behavior by overriding in
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > AbstractUdfStreamOperator
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > @Override
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > return userFunction instanceof CheckpointListener;
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > }
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > This idea can be further refined in also adding
> > >
> > > > requiresFinalCheckpoint
> > >
> > > >
> > >
> > > > > > to
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > CheckpointListener to exclude all operators with UDFs that
> > >
> > > implement
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > CheckpointListener but do not need it for 2pc.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > @Override
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > return userFunction instanceof CheckpointListener &&
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > ((CheckpointListener) userFunction).requiresFinalCheckpoint();
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > }
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > That approach would also work for statebackends/snapshot
> > strategies
> > >
> > > >
> > >
> > > > > that
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > require some 2pc.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > If we can contain it to the @PublicEvolving StreamOperator, it
> > >
> > > would
> > >
> > > > be
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > better of course.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > Best,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > Arvid
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > wrote:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Hey,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > I would like to raise a concern about implementation of the
> > final
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > checkpoints taking into account operators/functions that are
> > >
> > > >
> > >
> > > > > > implementing
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > two phase commit (2pc) protocol for exactly-once processing
> > with
> > >
> > > > some
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > external state (kept outside of the Flink). Primarily
> > >
> > > exactly-once
> > >
> > > >
> > >
> > > > > > sinks.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > First of all, as I understand it, this is not planned in the
> > >
> > > first
> > >
> > > >
> > >
> > > > > > > version
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > of this FLIP. I'm fine with that, however I would strongly
> > >
> > > > emphasize
> > >
> > > >
> > >
> > > > > > this
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > in every place we will be mentioning FLIP-147 efforts. This
> is
> > >
> > > >
> > >
> > > > > because
> > >
> > > >
> > >
> > > > > > > me,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > as a user, upon hearing "Flink supports checkpointing with
> > >
> > > bounded
> > >
> > > >
> > >
> > > > > > > inputs"
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > I would expect 2pc to work properly and to commit the
> external
> > >
> > > side
> > >
> > > >
> > >
> > > > > > > effects
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > upon finishing. As it is now, I (as a user) would be
> surprised
> > >
> > > > with a
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > silent data loss (of not committed trailing data). This is
> > just a
> > >
> > > >
> > >
> > > > > > remark,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > that we need to attach this warning to every blog
> > >
> > > >
> > >
> > > > > > post/documentation/user
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > mailing list response related to "Support Checkpoints After
> > Tasks
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Finished". Also I would suggest to prioritize the follow up
> of
> > >
> > > >
> > >
> > > > > > supporting
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 2pc.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Secondly, I think we are missing how difficult and
> problematic
> > >
> > > will
> > >
> > > >
> > >
> > > > > be
> > >
> > > >
> > >
> > > > > > > 2pc
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > support with the final checkpoint.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > For starters, keep in mind that currently 2pc can be
> > implemented
> > >
> > > by
> > >
> > > >
> > >
> > > > > > users
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > using both `@Public` APIs as functions and `@PublicEvolving`
> > >
> > > >
> > >
> > > > > operators
> > >
> > > >
> > >
> > > > > > in
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > any place in the job graph. It's not limited to only the
> sinks.
> > >
> > > For
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > example users could easily implement the `AsynFunction` (for
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `AsyncWaitOperator`) that is using 2pc based on the
> > >
> > > >
> > >
> > > > > > `CheckpointListener`
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > interface. I'm not saying it's common, probably just a tiny
> > >
> > > > minority
> > >
> > > >
> > >
> > > > > of
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > users are doing that (if any at all), but nevertheless
> that's
> > >
> > > >
> > >
> > > > > possible
> > >
> > > >
> > >
> > > > > > > and
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > currently (implicitly?) supported in Flink.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Next complication is the support of bounded streams
> > >
> > > >
> > >
> > > > > (`BoundedOneInput`
> > >
> > > >
> > >
> > > > > > or
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `BoundedMultiInput` interfaces) and the closing/shutdown
> > >
> > > procedure
> > >
> > > > of
> > >
> > > >
> > >
> > > > > > the
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > operators. Currently it works as follows:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 1. `endOfInput` is called on the first operator in the chain
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 2. We quiesce the processing timers
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> > for
> > >
> > > > the
> > >
> > > >
> > >
> > > > > > > first
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > operator, so no new timers will be triggered
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 3. We wait for the already fired timers to finish executing
> > >
> > > > (spinning
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > mailbox loop)
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 4. We are closing the first operator
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 5. We go to the next (second) operator in the chain and
> repeat
> > >
> > > the
> > >
> > > >
> > >
> > > > > > steps
> > >
> > > >
> > >
> > > > > > > 1.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > to 5.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > This is because operators can emit data after processing
> > >
> > > >
> > >
> > > > > `endOfInput`,
> > >
> > > >
> > >
> > > > > > > from
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > timers, async mailbox actions and inside the `close` method
> > >
> > > itself.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Now the problem is to support the final checkpoint with 2pc,
> we
> > >
> > > > need
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > trigger `snapshotState` and `notifyCheckpointComplete` call
> at
> > >
> > > the
> > >
> > > >
> > >
> > > > > very
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > least only after `endOfInput` call on the operator. Probably
> > the
> > >
> > > > best
> > >
> > > >
> > >
> > > > > > > place
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > would be in between steps 3. and 4. However that means, we
> > would
> > >
> > > be
> > >
> > > >
> > >
> > > > > > > forced
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > to wait for steps 1. to 3. to finish, then wait for a next
> > >
> > > > checkpoint
> > >
> > > >
> > >
> > > > > > to
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > trigger AND complete, before finally closing the head
> operator,
> > >
> > > and
> > >
> > > >
> > >
> > > > > > only
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > then we can start closing the next operator in the chain:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 1. `endOfInput` is called on the first operator in the chain
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 2. We quiesce the processing timers
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> > for
> > >
> > > > the
> > >
> > > >
> > >
> > > > > > > first
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > operator, so no new timers will be triggered
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 3. We wait for the already fired timers to finish executing
> > >
> > > > (spinning
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > mailbox loop)
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > *3b. We wait for one more checkpoint to trigger and for the
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `notifyCheckpointComplete` RPC.*
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 4. We are closing the first operator
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 5. We go to the next (second) operator in the chain and
> repeat
> > >
> > > the
> > >
> > > >
> > >
> > > > > > steps
> > >
> > > >
> > >
> > > > > > > 1.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > to 5.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > That means, we can close one operator per successful
> > checkpoint.
> > >
> > > To
> > >
> > > >
> > >
> > > > > > close
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 10 operators, we would need 10 successful checkpoints.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > I was thinking about different approaches to this problem,
> and
> > I
> > >
> > > >
> > >
> > > > > > couldn't
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > find any viable ones. All I could think of would break the
> > >
> > > current
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `@Public` API and/or would be ugly/confusing for the users.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > For example a relatively simple solution, to introduce a
> > >
> > > `preClose`
> > >
> > > >
> > >
> > > > > or
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `flush` method to the operators, with a contract that after
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `flush`, operators would be forbidden from emitting more
> > records,
> > >
> > > > so
> > >
> > > >
> > >
> > > > > > that
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > we can replace step 4. with this `flush` call, and then
> having
> > a
> > >
> > > >
> > >
> > > > > single
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > checkpoint to finish 2pc for all of the operators inside the
> > >
> > > chain,
> > >
> > > >
> > >
> > > > > > > doesn't
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > work. Sheer fact of adding this `flush` method and changing
> the
> > >
> > > >
> > >
> > > > > > contract
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > would break the current API and Yun Gao has pointed out to
> me,
> > >
> > > that
> > >
> > > >
> > >
> > > > > we
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > either already support, or want to support operators that
> are
> > >
> > > >
> > >
> > > > > emitting
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > records from within the `notifyCheckpointComplete` call:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Yun Gao:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > like with the new sink api there might be writer ->
> committer
> > >
> > > ->
> > >
> > > >
> > >
> > > > > > global
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > committer, the committer would need to wait for the last
> > >
> > > checkpoint
> > >
> > > >
> > >
> > > > > to
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > commit
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > the last piece of data, and after that it also need to
> emit
> > the
> > >
> > > >
> > >
> > > > > list
> > >
> > > >
> > >
> > > > > > of
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > transactions get committed to global committer to do some
> > >
> > > >
> > >
> > > > > finalization
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > logic.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > So it wouldn't solve the problem (at least not fully).
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > I don't know if anyone has any better ideas how to solve
> this
> > >
> > > >
> > >
> > > > > problem?
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Piotrek
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > pt., 15 sty 2021 o 14:57 Yun Gao
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > napisał(a):
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Hi Aljoscha,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > I think so since we seems to do not have other divergence
> and
> > >
> > > new
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > objections now. I'll open the vote then. Very thanks!
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Best,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Yun
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > > ------------------------------------------------------------------
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > From:Aljoscha Krettek
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Send Time:2021 Jan. 15 (Fri.) 21:24
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > To:dev
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After
> > Tasks
> > >
> > > >
> > >
> > > > > > > Finished
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Thanks for the summary! I think we can now move towards a
> > >
> > > [VOTE]
> > >
> > > >
> > >
> > > > > > > thread,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > right?
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > On 2021/01/15 13:43, Yun Gao wrote:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >1) For the problem that the "new" root task coincidently
> > >
> > > > finished
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >before getting triggered successfully, we have listed two
> > >
> > > > options
> > >
> > > >
> > >
> > > > > in
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >the FLIP-147[1], for the first version, now we are not
> tend
> > to
> > >
> > > > go
> > >
> > > >
> > >
> > > > > > with
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >the first option that JM would re-compute and re-trigger
> new
> > >
> > > >
> > >
> > > > > sources
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >when it realized some tasks are not triggered
> successfully.
> > >
> > > This
> > >
> > > >
> > >
> > > > > > > option
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >would avoid the complexity of adding new PRC and
> duplicating
> > >
> > > > task
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >states, and in average case it would not cause too much
> > >
> > > > overhead.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > You wrote "we are *not* tend to go with the first option",
> > but
> > >
> > > I
> > >
> > > >
> > >
> > > > > > think
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > you meant wo write "we tend to *now* go with the first
> > option",
> > >
> > > >
> > >
> > > > > > right?
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > That's also how it is in the FLIP, I just wanted to
> clarify
> > for
> > >
> > > > the
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > mailing list.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > >
> > >
> > >
> >
>
>

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

Posted by Till Rohrmann <tr...@apache.org>.
I think you are right with the problem of endOfInput. endOfInput should not
be used to commit final results. In fact if this termination fails then we
might end up in a different outcome of the job which is equally valid as
the one before the failure.

Concerning unaligned checkpoints, I think they don't play well together
with draining a streaming pipeline. The problem is that in the draining
case you want to process all records which are still in flight but
unaligned checkpoints don't guarantee this as they can jump in flight
records.

I think the MAX_WATERMARK event should be sent either just before or with
the EndOfPartitionEvent. If we ensure that the MAX_WATERMARK is only
persisted in state if a recovery will trigger immediately the shut down of
this operator, then it shouldn't be an issue.

Cheers,
Till

On Sun, Feb 28, 2021 at 9:06 AM Kezhu Wang <ke...@gmail.com> wrote:

> Hi Till,
>
> Just for bookkeeping, some observations from current implementation.
>
> > With this model, the final checkpoint is quite simple because it is
> ingrained in the lifecycle of an operator. Differently said an operator
> will only terminate after it has committed its side effects and seen the
> notifyCheckpointComplete message (if it is stateful).
>
> Currently, we could not mark this operator(or subtask) as terminated since
> result of `notifyCheckpointComplete`(possible side effect committing) is
> not taken into account of the belonging checkpoint. The job has to run to
> next safe point(finished or next checkpoint success) to be marked as
> “terminated”.
>
> > How to implement the stop-with-savepoint --drain/terminate command with
> this model: One idea could be to tell the sources that they should stop
> reading. This should trigger the EndOfPartitionEvent to be sent
> downstream.
> This will transition all operators into the TERMINATING state.
>
> Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
> returned. To achieve above, possible works should be required:
> * Promote  `EndOfPartitionEvent` from `Task` to `StreamTask`. This may
> have some interferences with BatchTask or network io stack.
> * Or introducing stream task level `EndOfUserRecordsEvent`(from PR#14831
> @Yun @Piotr)
> * Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.
>
> Besides this, I would like to quote some discussion from FLINK-21467
> between @Piotr and me:
>
> From @Piotr
> > Note, that it's not only that endOfInput can be called multiple times.
> There is a very remote possibility that the following scenario will happen:
> 1. checkpoint is taken (successfully)
> 2. sources are finishing
> 3. endOfInput is issued
> 4. job fails
> 5. job restarts to checkpoint 1.
> 6. after failover, because of some non deterministic logic in the source,
> sources are not finishing
>
> From me
> > But I think there is little work Flink can do to cope with this kind of
> issues. The checkpoint could be a savepoint triggered from user side and
> the "non deterministic logic" could be a change from user(eg. changing of
> stoppingOffsets in KafkaSource).
>
> > I think the "non deterministic logic" could cause trouble in combination
> with unaligned checkpoint and downstream window operator. Unaligned
> checkpoint will persist "MAX_WATERMARK" in state, after restarting,
> "MAX_WATERMARK" will cause downstream window operator ignores all future
> inputs.
>
> FLIP-147 demands no new records from end-of-stream-flushing, but source
> will emit “MAX_WATERMARK” on ending. Previously, I thought it is not a
> valid issue, but turn out that it could cause trouble under scenario listed
> by @Piotr if I am not wrong.
>
>
> PR#14831: https://github.com/apache/flink/pull/14831
> FLINK-21467: https://issues.apache.org/jira/browse/FLINK-21467
>
>
> Best,
> Kezhu Wang
>
> On February 27, 2021 at 18:12:20, Till Rohrmann (trohrmann@apache.org)
> wrote:
>
> Thanks for all your thoughts. I think we should further think through
> whether to allow checkpoints after an operator has emitted all its records
> (e.g. after close is called currently) or not. I think by doing this we
> would nicely decouple the checkpoint taking from the operator lifecycle
> and
> wouldn't need special checkpoints/savepoints for the final checkpoint and
> stop-with-savepoint --drain. Let me try to explain this a bit more
> detailed.
>
> If we say an operator has the rough lifecycle RUNNING => TERMINATING =>
> TERMINATED where we go from RUNNING into TERMINATING after we have seen
> the
> EndOfPartitionEvent and flushed all our records. The operator goes from
> TERMINATING => TERMINATED if it has persisted all its possible side
> effects. Throughout all states, it is possible to trigger a checkpoint. A
> stateless operator will immediately go from TERMINATING to TERMINATED
> whereas a stateful operator would wait for another checkpoint to be
> triggered and successfully completed (notifyCheckpointComplete).
>
> With this model, the final checkpoint is quite simple because it is
> ingrained in the lifecycle of an operator. Differently said an operator
> will only terminate after it has committed its side effects and seen the
> notifyCheckpointComplete message (if it is stateful). Here it is important
> to note that in the streaming case, different bounded operators can
> terminate at different times. They don't have to terminate all with the
> same checkpoint.
>
> How to implement the stop-with-savepoint --drain/terminate command with
> this model: One idea could be to tell the sources that they should stop
> reading. This should trigger the EndOfPartitionEvent to be sent
> downstream.
> This will transition all operators into the TERMINATING state. Next the JM
> can trigger a checkpoint to shut the operators down and not to wait for
> the
> next automatic checkpoint trigger event.
>
> By allowing checkpoints throughout the entire lifecycle of an operator we
> disallow sending records from notifyCheckpointComplete because this
> message
> will also be sent in the state TERMINATING where an operator has already
> produced all of its records.
>
> What do you think? Maybe this model overlooks some important cases. One
> downside is that we will break the operator API with changing the
> lifecycle
> of an operator.
>
> Cheers,
> Till
>
>
>
> On Fri, Feb 26, 2021 at 7:29 AM Kezhu Wang <ke...@gmail.com> wrote:
>
> > Hi Yun,
> >
> > The termination phase I drafted depends on a de-fact that
> > `notifyCheckpointComplete` is ignored currently after `close`. Thus, any
> > external materialization inside `close`(including `endInput`) or
> > `notifyCheckpointComplete` is either not guaranteed or not committed.
> >
> > I tried to emphasize no-breaking changes and post-pone migration in
> later
> > releases for this termination phase. But now, I am kind of worry about
> > whether it will cause long-term maintenance hell.
> >
> > Personally, I think allowing checkpoint after `close`(@Till proposed
> this
> > in FLINK-21133) could be the minimal change. But there are concerns:
> > * It will break some existing code possibly in a silent way.(@Piotr
> pointed
> > this already in FLINK-21133)
> > * I think the name `close` is kind of misleading. (@Piotr suggested
> > renaming this to `finish` to not break code silently in FLINK-21133)
> >
> > > stop-with-savepoint --drain would wait for a specific savepoint
> >
> > For stop-with-savepoint, the checkpoint barrier is already created or
> > received there.
> >
> >
> > Best,
> > Kezhu Wang
> >
> > On February 25, 2021 at 21:02:30, Yun Gao (yungao.gy@aliyun.com) wrote:
> >
> > Hi all,
> >
> > Very thanks for the discussions!
> >
> >
> >
> > A. Regarding how to avoid emitting records in notifyCheckpointComplete:
> >
> > Currently the structure of a new sink is writer -> committer -> global
> > committer and the paralellism of global committer
> > must be one. By design it would be used in several cases:
> > 1. writer -> committer: for normal sinks that write pending data in
> writer
> > and commit pending data in committer.
> > 2. writer -> global committer: for sinks require the committer's
> > parallelism be 1, like iceberg sink which
> > relies on optimistic lock to commit data hopes to reduce the conflicts.
> > 3. writer -> committer -> global committer: for sinks that also need to
> > write meta data (like _SUCCESS file
> > or add record in hive metastore) in global committer.
> >
> > The third case would cause the cascade commit problem. To overcome this
> > problem, we would like to
> > 1. Previously the global committer also support writing metadata with
> 2pc.
> > Now we disable this mode
> > and always rely on the property that writing metadata is repeatable.
> With
> > this limitation there should no
> > need of cascade commit, but the committer would still emit records in
> > notifyCheckpointComplete.
> > 2. We further move global committer in the case 3 to the operator
> > coordinator. Then the committer would
> > not need to emit records, but communicates with its operator
> coordinator.
> >
> > One core issue for using the OperatorCoordinator is how to keep the
> > communication between the operator
> > and the OperatorCoordinator exactly-once. Since the message is always
> from
> > the oeprator to the OperatorCoordinator
> > in this case, we would only need to bookkeep the message sent between
> the
> > OperatorCoordinator takes snapshot and
> > the Operator takes snapshot in the state of the Operator.
> >
> > On how to achieve the change in detail we would still need some think,
> it
> > currently seems we would have to had
> > some modification to the current new sink api.
> >
> >
> >
> > B. Regarding the stop-with-savepoint --drain
> >
> > Very thanks @Piotr for the further explanation and now I realize I have
> > understand wrongly for the semantics of
> > stop-with-savepoint --drain. Now I think that the problem should be we
> > should also include the records produced in
> > `endOfInput()` and `close()` also in the last savepoint, am I correct?
> If
> > so, it seems we still have some undetermined options for
> > the lifecycle of the operator, like in Kezhu's proposal the close()
> happens
> > at last, but it seems close() might also emit records (
> > so now the operator are closed with op1's close() -> op2's endOfInput()
> ->
> > op2's close() -> op3's endOfinput -> ...) ?
> >
> > And on the other side, as Kezhu has also proposed, perhapse we might
> have
> > the stop-with-savepoint --drain and normal exit in the same process,
> > but have slightly difference in that stop-with-savepoint --drain would
> wait
> > for a specific savepoint and in normal exit, the operator
> > might wait for arbitrary checkpoint. If we could achieve not emitting
> > records in notifyCheckpointComplete, stop-with-savepoint --drain could
> > be done with one savepoint, and for the normal exit, the operator would
> not
> > need to wait for other slow operators to exit.
> >
> > Best,
> > Yun
> >
> >
> >
> > ------------------Original Mail ------------------
> > *Sender:*Kezhu Wang <ke...@gmail.com>
> > *Send Date:*Thu Feb 25 15:11:53 2021
> > *Recipients:*Flink Dev <de...@flink.apache.org>, Piotr Nowojski <
> > piotr.nowojski@gmail.com>
> > *CC:*Guowei Ma <gu...@gmail.com>, jingsonglee0@gmail.com <
> > jingsonglee0@gmail.com>
> > *Subject:*Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > Finished
> >
> > > Hi all, thanks for driving this and especially Piotr for re-active
> this
> > >
> > > thread.
> > >
> > >
> > >
> > > First, for `notifyCheckpointComplete`, I have strong preference
> towards
> > >
> > > "shut down the dataflow
> > >
> > > pipeline with one checkpoint in total", so I tend to option dropping
> > "send
> > >
> > > records" from
> > >
> > > `notifyCheckpointComplete` for next level committing in pipeline, if
> we
> > >
> > > ever support it. Without this,
> > >
> > > we are unable to stop a pipeline manually with all results
> materialized.
> > >
> > >
> > >
> > > Second, for shutdown unification of `stop-with-savepoint --drain` and
> > >
> > > FLIP-147, I draft following
> > >
> > > phase based on emerging proposals(mainly by Piotr and Till) in this
> > thread
> > >
> > > and FLINK-21133.
> > >
> > >
> > >
> > > ```java
> > >
> > > // StreamTask termination phase
> > >
> > > finish()(Call StreamOperator.finish in chaining order)
> > >
> > > advanceToEndOfEventTime()(nop if for no-source inputs, this could also
> be
> > >
> > > done in input processor or finish ?)
> > >
> > > if (there-is-a-pending-terminate-savepoint) {
> > >
> > > triggerBarrierForDownStream();
> > >
> > > waitCheckpointComplete();
> > >
> > > } else if (require-two-phase-commit-in-shutdown) {
> > >
> > > waitFinalCheckpoint();
> > >
> > > waitCheckpointComplete();
> > >
> > > }
> > >
> > > // TODO: Ignore/Cancel all checkpoints from this point(or hit to
> > >
> > > coordinator in final checkpoint response ?).
> > >
> > > close();
> > >
> > > ```
> > >
> > >
> > >
> > > Branches in `if` could reside in different code paths, but the effect
> > >
> > > should be similar to above.
> > >
> > > The divergence reflects fact that "stop-with-savepoint --drain" is
> > >
> > > triggered already while we need
> > >
> > > to wait for final checkpoint in natural input exhausting.
> > >
> > >
> > >
> > > With carefully chosen default functions, we will not break existing
> > >
> > > interfaces.
> > >
> > > * `StreamOperator.finish`
> > >
> > > * `RichFunction.finish`
> > >
> > >
> > >
> > > For existing codes:
> > >
> > > 1. "stop-with-savepoint" with no migration, it should behaves as
> before.
> > >
> > > 2. "require-two-phase-commit-in-shutdown" evaluates to false and no
> > >
> > > migration, it is same as before.
> > >
> > > 3. "require-two-phase-commit-in-shutdown" evaluates to true and no
> > >
> > > migration, records in `close` will
> > >
> > > lose.
> > >
> > >
> > >
> > > For case#3:
> > >
> > > A. No external materialization in `StreamOperator.close`: Nothing to
> > lose.
> > >
> > > B. External materialization only in `StreamOperator.close`: I don't
> think
> > >
> > > Flink ever claimed that there is
> > >
> > > any guarantee for this situation.
> > >
> > > C. External materialization in `notifyCheckpointComplete` from
> > >
> > > `StreamOperator.close`: But due to fact that
> > >
> > > `notifyCheckpointComplete` was ignored after operator
> > >
> > > closed(FLINK-16383), so there will be no external
> > >
> > > materialization to lose.
> > >
> > >
> > >
> > > Then, we could recommend users to migrate possible
> > “end-of-stream-flushing”
> > >
> > > from “close" to “finish”.
> > >
> > >
> > >
> > > Best,
> > >
> > > Kezhu Wang
> > >
> > >
> > >
> > >
> > >
> > > On February 24, 2021 at 23:47:36, Piotr Nowojski (
> > piotr.nowojski@gmail.com
> > > )
> > >
> > > wrote:
> > >
> > >
> > >
> > > Thanks for the reponses Guowei and Yun,
> > >
> > >
> > >
> > > Could you elaborate more/remind me, what does it mean to replace
> emitting
> > >
> > > results from the `notifyCheckpointComplete` with `OperatorCoordinator`
> > >
> > > approach?
> > >
> > >
> > >
> > > About the discussion in FLINK-21133 and how it relates to FLIP-147.
> You
> > are
> > >
> > > right Yun gao, that in case of `stop-with-savepoint --drain` the whole
> > job
> > >
> > > finishes, while in FLIP-147 some of the sources can work for
> arbitrarily
> > >
> > > long time after others have already finished. However from the runtime
> > >
> > > perspective, especially how to close a Task and it's operators, there
> is
> > no
> > >
> > > difference between those two cases. In both cases, we need to end
> input,
> > >
> > > shut down processing time timers and commit external side effects (two
> > >
> > > phase commit support) before exiting the task. That's how the
> discussion
> > >
> > > about the behaviour of "stop-with-savepoint" was connected with
> FLIP-147.
> > >
> > >
> > >
> > > Currently on master, "stop-with-savepoint --drain" drains/flushes
> > buffered
> > >
> > > records and deals correctly with timers, but all of that happens AFTER
> > >
> > > savepoint was completed. So any records flushed from the operators
> during
> > >
> > > endOfInput/close/shutting down processing timers are never committed
> to
> > >
> > > external systems. This is exactly the same problem as the "two phase
> > >
> > > commit" problem of FLIP-147, that should have the same solution and it
> > >
> > > should be solved at the same time. For example if we go with the *very
> > >
> > > undesirable* "one closed operator per one completed
> > checkpoint/savepoint",
> > >
> > > in both cases CheckpointCoordinator, Scheduler and Task would need to
> > keep
> > >
> > > the task alive and keep triggering checkpoints for that task, until
> all
> > >
> > > operators in the operator chain are closed (one closed operator per
> one
> > >
> > > completed checkpoint).
> > >
> > >
> > >
> > > Piotrek
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > > śr., 24 lut 2021 o 11:30 Yun Gao napisał(a):
> > >
> > >
> > >
> > > > Hi Till, Guowei,
> > >
> > > >
> > >
> > > > Very thanks for initiating the disucssion and the deep thoughts!
> > >
> > > >
> > >
> > > > For the notifyCheckpointComplete, I also agree we could try to avoid
> > >
> > > > emitting new records in notifyCheckpointComplete via using
> > >
> > > > OperatorCoordinator
> > >
> > > > for new sink API. Besides, the hive sink might also need some
> > >
> > > modification
> > >
> > > > for it also emits records in notifyCheckpointComplete.
> > >
> > > >
> > >
> > > > For unifying the process of stopping with savepoint and finished due
> to
> > >
> > > > all records
> > >
> > > > are processed, I also agree with that unifying would always be
> better
> > if
> > >
> > > > we could achieve,
> > >
> > > > but I'm still not fully catch up with the implementation: Based on
> the
> > >
> > > > discussion in FLINK-21133,
> > >
> > > > my understanding is that for stopping with savepoint, now we want to
> > >
> > > first
> > >
> > > > stop the source, then we
> > >
> > > > trigger a savepoint, and after the source received
> > >
> > > > notifyCheckpointComplete, the source would
> > >
> > > > start emitting EndOfPartitionEvent to finish the job, am I correct ?
> > >
> > > >
> > >
> > > > For normal finish, a difference to me might be if we have multiple
> > >
> > > > sources, we could not guarantee
> > >
> > > > when the sources are to finish. We might have one source run one 1
> > minute
> > >
> > > > and another one run for
> > >
> > > > 1 hour. To unify with the process with stop with savepoint, we might
> > need
> > >
> > > > to hold the fast source until
> > >
> > > > all the sources are finished? An coordinator would be introduced to
> > count
> > >
> > > > the number of sources
> > >
> > > > runing and trigger the final savepoint / checkpoint. For the extreme
> > >
> > > > cases, if we have both bounded and
> > >
> > > > unbounded sources, we might only count how much bounded source are
> > >
> > > > remaining ? And if all the bounded
> > >
> > > > sources are finished we would trigger the special checkpoint. After
> all
> > >
> > > > the bounded part of the graph are
> > >
> > > > finished, the the remaining part could still do checkpoint and
> commit
> > >
> > > data
> > >
> > > > with FLIP-147.
> > >
> > > >
> > >
> > > > Best,
> > >
> > > > Yun
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > ------------------Original Mail ------------------
> > >
> > > > Sender:Guowei Ma
> > >
> > > > Send Date:Wed Feb 24 17:35:36 2021
> > >
> > > > Recipients:dev
> > >
> > > > CC:Arvid Heise
> > >
> > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > Finished
> > >
> > > > Hi, Till
> > >
> > > >
> > >
> > > > Thank you very much for your careful consideration
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > *1. Emit records in `NotifyCheckpointComplete`.*
> > >
> > > >
> > >
> > > > Sorry for making you misunderstanding because of my expression. I
> just
> > >
> > > >
> > >
> > > > want to say the current interface does not prevent users from doing
> it.
> > >
> > > >
> > >
> > > > From the perspective of the new sink api, we might not depend on
> > emitting
> > >
> > > >
> > >
> > > > records in `NotifyCheckpointComplete`, like using
> `OperatorCoordinator`
> > >
> > > >
> > >
> > > > instead.
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > *2. What does the FLIP-147 guarantee?*I think initially this FLIP
> want
> > to
> > >
> > > >
> > >
> > > > achieve two targets:
> > >
> > > >
> > >
> > > > 1. Tasks/Operators exit correctly (as you mentioned the lifecycle of
> a
> > >
> > > >
> > >
> > > > Task/StreamTask/StreamOperator.).
> > >
> > > >
> > >
> > > > 2. Continue to trigger checkpoint after some tasks for mixed jobs.
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > I think the first thing is related to the discussion in FLINK-21133.
> > If I
> > >
> > > >
> > >
> > > > understand correctly, in addition to supporting the tasks /
> operators
> > to
> > >
> > > >
> > >
> > > > exit correctly, now we also want to unify the process of the tasks
> and
> > >
> > > >
> > >
> > > > operators for savepoint / finish.
> > >
> > > >
> > >
> > > > I think the second thing is orthogonal to the FLINK-21133 because
> there
> > >
> > > are
> > >
> > > >
> > >
> > > > topologies that have both the bounded and unbounded input.
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > *3. How to unify the operator exit process of FLIP-147 with
> > >
> > > >
> > >
> > > > stop-with-savepoint?*
> > >
> > > >
> > >
> > > > I am not very sure about how to do it yet. But if I understand the
> > >
> > > >
> > >
> > > > discussion in the jira correctly it needs to introduce some logic
> into
> > >
> > > >
> > >
> > > > `CheckpointCoordinator`, which responses for triggering “the unified
> > >
> > > >
> > >
> > > > operator exit process”. Am I correct?
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > Best,
> > >
> > > >
> > >
> > > > Guowei
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann wrote:
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > > Thanks for the explanation Yun and Guowei. I have to admit that I
> do
> > >
> > > not
> > >
> > > >
> > >
> > > > > fully understand why this is strictly required but I think that we
> > are
> > >
> > > >
> > >
> > > > > touching two very important aspects which might have far fetching
> > >
> > > >
> > >
> > > > > consequences for how Flink works:
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > 1) Do we want to allow that multiple checkpoints are required to
> > >
> > > >
> > >
> > > > > materialize results?
> > >
> > > >
> > >
> > > > > 2) Do we want to allow to emit records in
> notifyCheckpointComplete?
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > For 1) I am not sure whether this has been discussed within the
> > >
> > > community
> > >
> > > >
> > >
> > > > > sufficiently. Requiring multiple checkpoints to materialize a
> result
> > >
> > > >
> > >
> > > > > because of multi level committers has the consequence that we
> > increase
> > >
> > > > the
> > >
> > > >
> > >
> > > > > latency from checkpoint interval to #levels * checkpoint interval.
> > >
> > > >
> > >
> > > > > Moreover, having to drain the pipeline in multiple steps, would
> break
> > >
> > > the
> > >
> > > >
> > >
> > > > > stop-with-savepoint --drain because which savepoint do you report
> to
> > >
> > > the
> > >
> > > >
> > >
> > > > > user?
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > For 2) allowing to send records after the final
> > >
> > > notifyCheckpointComplete
> > >
> > > >
> > >
> > > > > will effectively mean that we need to shut down a topology in
> > multiple
> > >
> > > >
> > >
> > > > > steps (in the worst case one operator per checkpoint). This would
> be
> > a
> > >
> > > >
> > >
> > > > > strong argument for not allowing this to me. The fact that users
> can
> > >
> > > send
> > >
> > > >
> > >
> > > > > records after the notifyCheckpointComplete is more by accident
> than
> > by
> > >
> > > >
> > >
> > > > > design. I think we should make this a very deliberate decision and
> in
> > >
> > > > doubt
> > >
> > > >
> > >
> > > > > I would be in favour of a more restrictive model unless there is a
> > very
> > >
> > > >
> > >
> > > > > good reason why this should be supported.
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > Taking also the discussion in FLINK-21133 [1] into account, it
> seems
> > to
> > >
> > > > me
> > >
> > > >
> > >
> > > > > that we haven't really understood what kind of guarantees we want
> to
> > >
> > > give
> > >
> > > >
> > >
> > > > > to our users and how the final checkpoint should exactly work. I
> > >
> > > > understand
> > >
> > > >
> > >
> > > > > that this is not included in the first scope of FLIP-147 but I
> think
> > >
> > > this
> > >
> > > >
> > >
> > > > > is so important that we should figure this out asap. Also because
> the
> > >
> > > > exact
> > >
> > > >
> > >
> > > > > shut down behaviour will have to be aligned with the lifecycle of
> a
> > >
> > > >
> > >
> > > > > Task/StreamTask/StreamOperator. And last but not least because
> other
> > >
> > > >
> > >
> > > > > features such as the new sink API start building upon a shut down
> > model
> > >
> > > >
> > >
> > > > > which has not been fully understood/agreed upon.
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > [1] https://issues.apache.org/jira/browse/FLINK-21133
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > Cheers,
> > >
> > > >
> > >
> > > > > Till
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma wrote:
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > > > > Thanks Yun for the detailed explanation.
> > >
> > > >
> > >
> > > > > > A simple supplementary explanation about the sink case: Maybe we
> > >
> > > could
> > >
> > > >
> > >
> > > > > use
> > >
> > > >
> > >
> > > > > > `OperatorCoordinator` to avoid sending the element to the
> > downstream
> > >
> > > >
> > >
> > > > > > operator.
> > >
> > > >
> > >
> > > > > > But I agree we could not limit the users not to emit records in
> the
> > >
> > > >
> > >
> > > > > > `notiyCheckpointComplete`.
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > > > Best,
> > >
> > > >
> > >
> > > > > > Guowei
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > > > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao
> > >
> > > >
> > >
> > > > > > wrote:
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > > > > Hi all,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > I'd like to first detail the issue with emitting records in
> > >
> > > >
> > >
> > > > > > > notifyCheckpointComplete for context. For specific usage,
> > >
> > > >
> > >
> > > > > > > an example would be for sink, it might want to write some
> > metadata
> > >
> > > >
> > >
> > > > > after
> > >
> > > >
> > >
> > > > > > > all the transactions are committed
> > >
> > > >
> > >
> > > > > > > (like write a marker file _SUCCESS to the output directory).
> This
> > >
> > > > case
> > >
> > > >
> > >
> > > > > is
> > >
> > > >
> > >
> > > > > > > currently supported via the two level
> > >
> > > >
> > >
> > > > > > > committers of the new sink API: when received endOfInput(),
> the
> > >
> > > >
> > >
> > > > > Committer
> > >
> > > >
> > >
> > > > > > > wait for another checkpoint to
> > >
> > > >
> > >
> > > > > > > commits all the pending transactions and emit the list of
> files
> > to
> > >
> > > > the
> > >
> > > >
> > >
> > > > > > > GlobalCommitter. The GlobalCommitter
> > >
> > > >
> > >
> > > > > > > would wait for another checkpoint to also write the metadata
> with
> > >
> > > 2pc
> > >
> > > >
> > >
> > > > > > > (Although sometimes 2pc is not needed
> > >
> > > >
> > >
> > > > > > > for writing metadata, it should be only an optimization and
> still
> > >
> > > >
> > >
> > > > > > requires
> > >
> > > >
> > >
> > > > > > > the Committer do commit before
> > >
> > > >
> > >
> > > > > > > notifying the global Committer. Also another note is
> > >
> > > GlobalCommitter
> > >
> > > > is
> > >
> > > >
> > >
> > > > > > > also added for some other cases
> > >
> > > >
> > >
> > > > > > > like some sinks want an commiter with dop = 1, like
> IceBergSink).
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > However, a more general issue to me is that currently we do
> not
> > >
> > > limit
> > >
> > > >
> > >
> > > > > > > users to not emit records in
> > >
> > > >
> > >
> > > > > > > notifyCheckpointComplete in the API level. The sink case could
> be
> > >
> > > >
> > >
> > > > > viewed
> > >
> > > >
> > >
> > > > > > > as a special case, but in addition
> > >
> > > >
> > >
> > > > > > > to this one, logically users could also implement their own
> cases
> > >
> > > > that
> > >
> > > >
> > >
> > > > > > > emits records in notifyCheckpointComplete.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > Best,
> > >
> > > >
> > >
> > > > > > > Yun
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > ------------------Original Mail ------------------
> > >
> > > >
> > >
> > > > > > > Sender:Arvid Heise
> > >
> > > >
> > >
> > > > > > > Send Date:Fri Feb 12 20:46:04 2021
> > >
> > > >
> > >
> > > > > > > Recipients:dev
> > >
> > > >
> > >
> > > > > > > CC:Yun Gao
> > >
> > > >
> > >
> > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After
> Tasks
> > >
> > > >
> > >
> > > > > Finished
> > >
> > > >
> > >
> > > > > > > Hi Piotr,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > Thank you for raising your concern. Unfortunately, I do not
> have
> > a
> > >
> > > >
> > >
> > > > > better
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > idea than doing closing of operators intermittently with
> > >
> > > checkpoints
> > >
> > > > (=
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > multiple last checkpoints).
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > However, two ideas on how to improve the overall user
> experience:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > 1. If an operator is not relying on notifyCheckpointComplete,
> we
> > >
> > > can
> > >
> > > >
> > >
> > > > > > close
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > it faster (without waiting for a checkpoint). In general, I'd
> > >
> > > assume
> > >
> > > >
> > >
> > > > > that
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > almost all non-sinks behave that way.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > 2. We may increase the checkpointing frequency for the last
> > >
> > > >
> > >
> > > > > checkpoints.
> > >
> > > >
> > >
> > > > > > We
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > need to avoid overloading checkpoint storages and task
> managers,
> > >
> > > but
> > >
> > > > I
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > assume the more operators are closed, the lower the
> checkpointing
> > >
> > > >
> > >
> > > > > > interval
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > can be.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > For 1, I'd propose to add (name TBD):
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > default boolean StreamOperator#requiresFinalCheckpoint() {
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > return true;
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > }
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > This means all operators are conservatively (=slowly) closed.
> For
> > >
> > > > most
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > operators, we can then define their behavior by overriding in
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > AbstractUdfStreamOperator
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > @Override
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > return userFunction instanceof CheckpointListener;
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > }
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > This idea can be further refined in also adding
> > >
> > > > requiresFinalCheckpoint
> > >
> > > >
> > >
> > > > > > to
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > CheckpointListener to exclude all operators with UDFs that
> > >
> > > implement
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > CheckpointListener but do not need it for 2pc.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > @Override
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > return userFunction instanceof CheckpointListener &&
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > ((CheckpointListener) userFunction).requiresFinalCheckpoint();
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > }
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > That approach would also work for statebackends/snapshot
> > strategies
> > >
> > > >
> > >
> > > > > that
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > require some 2pc.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > If we can contain it to the @PublicEvolving StreamOperator, it
> > >
> > > would
> > >
> > > > be
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > better of course.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > Best,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > Arvid
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > wrote:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Hey,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > I would like to raise a concern about implementation of the
> > final
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > checkpoints taking into account operators/functions that are
> > >
> > > >
> > >
> > > > > > implementing
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > two phase commit (2pc) protocol for exactly-once processing
> > with
> > >
> > > > some
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > external state (kept outside of the Flink). Primarily
> > >
> > > exactly-once
> > >
> > > >
> > >
> > > > > > sinks.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > First of all, as I understand it, this is not planned in the
> > >
> > > first
> > >
> > > >
> > >
> > > > > > > version
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > of this FLIP. I'm fine with that, however I would strongly
> > >
> > > > emphasize
> > >
> > > >
> > >
> > > > > > this
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > in every place we will be mentioning FLIP-147 efforts. This
> is
> > >
> > > >
> > >
> > > > > because
> > >
> > > >
> > >
> > > > > > > me,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > as a user, upon hearing "Flink supports checkpointing with
> > >
> > > bounded
> > >
> > > >
> > >
> > > > > > > inputs"
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > I would expect 2pc to work properly and to commit the
> external
> > >
> > > side
> > >
> > > >
> > >
> > > > > > > effects
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > upon finishing. As it is now, I (as a user) would be
> surprised
> > >
> > > > with a
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > silent data loss (of not committed trailing data). This is
> > just a
> > >
> > > >
> > >
> > > > > > remark,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > that we need to attach this warning to every blog
> > >
> > > >
> > >
> > > > > > post/documentation/user
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > mailing list response related to "Support Checkpoints After
> > Tasks
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Finished". Also I would suggest to prioritize the follow up
> of
> > >
> > > >
> > >
> > > > > > supporting
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 2pc.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Secondly, I think we are missing how difficult and
> problematic
> > >
> > > will
> > >
> > > >
> > >
> > > > > be
> > >
> > > >
> > >
> > > > > > > 2pc
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > support with the final checkpoint.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > For starters, keep in mind that currently 2pc can be
> > implemented
> > >
> > > by
> > >
> > > >
> > >
> > > > > > users
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > using both `@Public` APIs as functions and `@PublicEvolving`
> > >
> > > >
> > >
> > > > > operators
> > >
> > > >
> > >
> > > > > > in
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > any place in the job graph. It's not limited to only the
> sinks.
> > >
> > > For
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > example users could easily implement the `AsynFunction` (for
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `AsyncWaitOperator`) that is using 2pc based on the
> > >
> > > >
> > >
> > > > > > `CheckpointListener`
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > interface. I'm not saying it's common, probably just a tiny
> > >
> > > > minority
> > >
> > > >
> > >
> > > > > of
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > users are doing that (if any at all), but nevertheless
> that's
> > >
> > > >
> > >
> > > > > possible
> > >
> > > >
> > >
> > > > > > > and
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > currently (implicitly?) supported in Flink.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Next complication is the support of bounded streams
> > >
> > > >
> > >
> > > > > (`BoundedOneInput`
> > >
> > > >
> > >
> > > > > > or
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `BoundedMultiInput` interfaces) and the closing/shutdown
> > >
> > > procedure
> > >
> > > > of
> > >
> > > >
> > >
> > > > > > the
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > operators. Currently it works as follows:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 1. `endOfInput` is called on the first operator in the chain
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 2. We quiesce the processing timers
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> > for
> > >
> > > > the
> > >
> > > >
> > >
> > > > > > > first
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > operator, so no new timers will be triggered
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 3. We wait for the already fired timers to finish executing
> > >
> > > > (spinning
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > mailbox loop)
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 4. We are closing the first operator
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 5. We go to the next (second) operator in the chain and
> repeat
> > >
> > > the
> > >
> > > >
> > >
> > > > > > steps
> > >
> > > >
> > >
> > > > > > > 1.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > to 5.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > This is because operators can emit data after processing
> > >
> > > >
> > >
> > > > > `endOfInput`,
> > >
> > > >
> > >
> > > > > > > from
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > timers, async mailbox actions and inside the `close` method
> > >
> > > itself.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Now the problem is to support the final checkpoint with 2pc,
> we
> > >
> > > > need
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > trigger `snapshotState` and `notifyCheckpointComplete` call
> at
> > >
> > > the
> > >
> > > >
> > >
> > > > > very
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > least only after `endOfInput` call on the operator. Probably
> > the
> > >
> > > > best
> > >
> > > >
> > >
> > > > > > > place
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > would be in between steps 3. and 4. However that means, we
> > would
> > >
> > > be
> > >
> > > >
> > >
> > > > > > > forced
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > to wait for steps 1. to 3. to finish, then wait for a next
> > >
> > > > checkpoint
> > >
> > > >
> > >
> > > > > > to
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > trigger AND complete, before finally closing the head
> operator,
> > >
> > > and
> > >
> > > >
> > >
> > > > > > only
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > then we can start closing the next operator in the chain:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 1. `endOfInput` is called on the first operator in the chain
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 2. We quiesce the processing timers
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> > for
> > >
> > > > the
> > >
> > > >
> > >
> > > > > > > first
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > operator, so no new timers will be triggered
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 3. We wait for the already fired timers to finish executing
> > >
> > > > (spinning
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > mailbox loop)
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > *3b. We wait for one more checkpoint to trigger and for the
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `notifyCheckpointComplete` RPC.*
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 4. We are closing the first operator
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 5. We go to the next (second) operator in the chain and
> repeat
> > >
> > > the
> > >
> > > >
> > >
> > > > > > steps
> > >
> > > >
> > >
> > > > > > > 1.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > to 5.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > That means, we can close one operator per successful
> > checkpoint.
> > >
> > > To
> > >
> > > >
> > >
> > > > > > close
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > 10 operators, we would need 10 successful checkpoints.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > I was thinking about different approaches to this problem,
> and
> > I
> > >
> > > >
> > >
> > > > > > couldn't
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > find any viable ones. All I could think of would break the
> > >
> > > current
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `@Public` API and/or would be ugly/confusing for the users.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > For example a relatively simple solution, to introduce a
> > >
> > > `preClose`
> > >
> > > >
> > >
> > > > > or
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `flush` method to the operators, with a contract that after
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > `flush`, operators would be forbidden from emitting more
> > records,
> > >
> > > > so
> > >
> > > >
> > >
> > > > > > that
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > we can replace step 4. with this `flush` call, and then
> having
> > a
> > >
> > > >
> > >
> > > > > single
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > checkpoint to finish 2pc for all of the operators inside the
> > >
> > > chain,
> > >
> > > >
> > >
> > > > > > > doesn't
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > work. Sheer fact of adding this `flush` method and changing
> the
> > >
> > > >
> > >
> > > > > > contract
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > would break the current API and Yun Gao has pointed out to
> me,
> > >
> > > that
> > >
> > > >
> > >
> > > > > we
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > either already support, or want to support operators that
> are
> > >
> > > >
> > >
> > > > > emitting
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > records from within the `notifyCheckpointComplete` call:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Yun Gao:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > like with the new sink api there might be writer ->
> committer
> > >
> > > ->
> > >
> > > >
> > >
> > > > > > global
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > committer, the committer would need to wait for the last
> > >
> > > checkpoint
> > >
> > > >
> > >
> > > > > to
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > commit
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > the last piece of data, and after that it also need to
> emit
> > the
> > >
> > > >
> > >
> > > > > list
> > >
> > > >
> > >
> > > > > > of
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > transactions get committed to global committer to do some
> > >
> > > >
> > >
> > > > > finalization
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > logic.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > So it wouldn't solve the problem (at least not fully).
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > I don't know if anyone has any better ideas how to solve
> this
> > >
> > > >
> > >
> > > > > problem?
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > Piotrek
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > pt., 15 sty 2021 o 14:57 Yun Gao
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > napisał(a):
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Hi Aljoscha,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > I think so since we seems to do not have other divergence
> and
> > >
> > > new
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > objections now. I'll open the vote then. Very thanks!
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Best,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Yun
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > > ------------------------------------------------------------------
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > From:Aljoscha Krettek
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Send Time:2021 Jan. 15 (Fri.) 21:24
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > To:dev
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After
> > Tasks
> > >
> > > >
> > >
> > > > > > > Finished
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > Thanks for the summary! I think we can now move towards a
> > >
> > > [VOTE]
> > >
> > > >
> > >
> > > > > > > thread,
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > right?
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > On 2021/01/15 13:43, Yun Gao wrote:
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >1) For the problem that the "new" root task coincidently
> > >
> > > > finished
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >before getting triggered successfully, we have listed two
> > >
> > > > options
> > >
> > > >
> > >
> > > > > in
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >the FLIP-147[1], for the first version, now we are not
> tend
> > to
> > >
> > > > go
> > >
> > > >
> > >
> > > > > > with
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >the first option that JM would re-compute and re-trigger
> new
> > >
> > > >
> > >
> > > > > sources
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >when it realized some tasks are not triggered
> successfully.
> > >
> > > This
> > >
> > > >
> > >
> > > > > > > option
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >would avoid the complexity of adding new PRC and
> duplicating
> > >
> > > > task
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > >states, and in average case it would not cause too much
> > >
> > > > overhead.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > You wrote "we are *not* tend to go with the first option",
> > but
> > >
> > > I
> > >
> > > >
> > >
> > > > > > think
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > you meant wo write "we tend to *now* go with the first
> > option",
> > >
> > > >
> > >
> > > > > > right?
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > That's also how it is in the FLIP, I just wanted to
> clarify
> > for
> > >
> > > > the
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > > mailing list.
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > > >
> > >
> > > >
> > >
> > > > > >
> > >
> > > >
> > >
> > > > >
> > >
> > > >
> > >
> > > >
> > >
> > >
> >
>
>

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

Posted by Kezhu Wang <ke...@gmail.com>.
Hi Till,

Just for bookkeeping, some observations from current implementation.

> With this model, the final checkpoint is quite simple because it is
ingrained in the lifecycle of an operator. Differently said an operator
will only terminate after it has committed its side effects and seen the
notifyCheckpointComplete message (if it is stateful).

Currently, we could not mark this operator(or subtask) as terminated since
result of `notifyCheckpointComplete`(possible side effect committing) is
not taken into account of the belonging checkpoint. The job has to run to
next safe point(finished or next checkpoint success) to be marked as
“terminated”.

> How to implement the stop-with-savepoint --drain/terminate command with
this model: One idea could be to tell the sources that they should stop
reading. This should trigger the EndOfPartitionEvent to be sent downstream.
This will transition all operators into the TERMINATING state.

Currently, EndOfPartitionEvent is only issued after StreamTask.invoke
returned. To achieve above, possible works should be required:
* Promote  `EndOfPartitionEvent` from `Task` to `StreamTask`. This may have
some interferences with BatchTask or network io stack.
* Or introducing stream task level `EndOfUserRecordsEvent`(from PR#14831
@Yun @Piotr)
* Or special handling of `CheckpointType.SAVEPOINT_TERMINATE`.

Besides this, I would like to quote some discussion from FLINK-21467
between @Piotr and me:

From @Piotr
> Note, that it's not only that endOfInput can be called multiple times.
There is a very remote possibility that the following scenario will happen:
1. checkpoint is taken (successfully)
2. sources are finishing
3. endOfInput is issued
4. job fails
5. job restarts to checkpoint 1.
6. after failover, because of some non deterministic logic in the source,
sources are not finishing

From me
> But I think there is little work Flink can do to cope with this kind of
issues. The checkpoint could be a savepoint triggered from user side and
the "non deterministic logic" could be a change from user(eg. changing of
stoppingOffsets in KafkaSource).

> I think the "non deterministic logic" could cause trouble in combination
with unaligned checkpoint and downstream window operator. Unaligned
checkpoint will persist "MAX_WATERMARK" in state, after restarting,
"MAX_WATERMARK" will cause downstream window operator ignores all future
inputs.

FLIP-147 demands no new records from end-of-stream-flushing, but source
will emit “MAX_WATERMARK” on ending. Previously, I thought it is not a
valid issue, but turn out that it could cause trouble under scenario listed
by @Piotr if I am not wrong.


PR#14831: https://github.com/apache/flink/pull/14831
FLINK-21467: https://issues.apache.org/jira/browse/FLINK-21467


Best,
Kezhu Wang

On February 27, 2021 at 18:12:20, Till Rohrmann (trohrmann@apache.org)
wrote:

Thanks for all your thoughts. I think we should further think through
whether to allow checkpoints after an operator has emitted all its records
(e.g. after close is called currently) or not. I think by doing this we
would nicely decouple the checkpoint taking from the operator lifecycle and
wouldn't need special checkpoints/savepoints for the final checkpoint and
stop-with-savepoint --drain. Let me try to explain this a bit more
detailed.

If we say an operator has the rough lifecycle RUNNING => TERMINATING =>
TERMINATED where we go from RUNNING into TERMINATING after we have seen the
EndOfPartitionEvent and flushed all our records. The operator goes from
TERMINATING => TERMINATED if it has persisted all its possible side
effects. Throughout all states, it is possible to trigger a checkpoint. A
stateless operator will immediately go from TERMINATING to TERMINATED
whereas a stateful operator would wait for another checkpoint to be
triggered and successfully completed (notifyCheckpointComplete).

With this model, the final checkpoint is quite simple because it is
ingrained in the lifecycle of an operator. Differently said an operator
will only terminate after it has committed its side effects and seen the
notifyCheckpointComplete message (if it is stateful). Here it is important
to note that in the streaming case, different bounded operators can
terminate at different times. They don't have to terminate all with the
same checkpoint.

How to implement the stop-with-savepoint --drain/terminate command with
this model: One idea could be to tell the sources that they should stop
reading. This should trigger the EndOfPartitionEvent to be sent downstream.
This will transition all operators into the TERMINATING state. Next the JM
can trigger a checkpoint to shut the operators down and not to wait for the
next automatic checkpoint trigger event.

By allowing checkpoints throughout the entire lifecycle of an operator we
disallow sending records from notifyCheckpointComplete because this message
will also be sent in the state TERMINATING where an operator has already
produced all of its records.

What do you think? Maybe this model overlooks some important cases. One
downside is that we will break the operator API with changing the lifecycle
of an operator.

Cheers,
Till



On Fri, Feb 26, 2021 at 7:29 AM Kezhu Wang <ke...@gmail.com> wrote:

> Hi Yun,
>
> The termination phase I drafted depends on a de-fact that
> `notifyCheckpointComplete` is ignored currently after `close`. Thus, any
> external materialization inside `close`(including `endInput`) or
> `notifyCheckpointComplete` is either not guaranteed or not committed.
>
> I tried to emphasize no-breaking changes and post-pone migration in later
> releases for this termination phase. But now, I am kind of worry about
> whether it will cause long-term maintenance hell.
>
> Personally, I think allowing checkpoint after `close`(@Till proposed this
> in FLINK-21133) could be the minimal change. But there are concerns:
> * It will break some existing code possibly in a silent way.(@Piotr
pointed
> this already in FLINK-21133)
> * I think the name `close` is kind of misleading. (@Piotr suggested
> renaming this to `finish` to not break code silently in FLINK-21133)
>
> > stop-with-savepoint --drain would wait for a specific savepoint
>
> For stop-with-savepoint, the checkpoint barrier is already created or
> received there.
>
>
> Best,
> Kezhu Wang
>
> On February 25, 2021 at 21:02:30, Yun Gao (yungao.gy@aliyun.com) wrote:
>
> Hi all,
>
> Very thanks for the discussions!
>
>
>
> A. Regarding how to avoid emitting records in notifyCheckpointComplete:
>
> Currently the structure of a new sink is writer -> committer -> global
> committer and the paralellism of global committer
> must be one. By design it would be used in several cases:
> 1. writer -> committer: for normal sinks that write pending data in
writer
> and commit pending data in committer.
> 2. writer -> global committer: for sinks require the committer's
> parallelism be 1, like iceberg sink which
> relies on optimistic lock to commit data hopes to reduce the conflicts.
> 3. writer -> committer -> global committer: for sinks that also need to
> write meta data (like _SUCCESS file
> or add record in hive metastore) in global committer.
>
> The third case would cause the cascade commit problem. To overcome this
> problem, we would like to
> 1. Previously the global committer also support writing metadata with
2pc.
> Now we disable this mode
> and always rely on the property that writing metadata is repeatable. With
> this limitation there should no
> need of cascade commit, but the committer would still emit records in
> notifyCheckpointComplete.
> 2. We further move global committer in the case 3 to the operator
> coordinator. Then the committer would
> not need to emit records, but communicates with its operator coordinator.
>
> One core issue for using the OperatorCoordinator is how to keep the
> communication between the operator
> and the OperatorCoordinator exactly-once. Since the message is always
from
> the oeprator to the OperatorCoordinator
> in this case, we would only need to bookkeep the message sent between the
> OperatorCoordinator takes snapshot and
> the Operator takes snapshot in the state of the Operator.
>
> On how to achieve the change in detail we would still need some think, it
> currently seems we would have to had
> some modification to the current new sink api.
>
>
>
> B. Regarding the stop-with-savepoint --drain
>
> Very thanks @Piotr for the further explanation and now I realize I have
> understand wrongly for the semantics of
> stop-with-savepoint --drain. Now I think that the problem should be we
> should also include the records produced in
> `endOfInput()` and `close()` also in the last savepoint, am I correct? If
> so, it seems we still have some undetermined options for
> the lifecycle of the operator, like in Kezhu's proposal the close()
happens
> at last, but it seems close() might also emit records (
> so now the operator are closed with op1's close() -> op2's endOfInput()
->
> op2's close() -> op3's endOfinput -> ...) ?
>
> And on the other side, as Kezhu has also proposed, perhapse we might have
> the stop-with-savepoint --drain and normal exit in the same process,
> but have slightly difference in that stop-with-savepoint --drain would
wait
> for a specific savepoint and in normal exit, the operator
> might wait for arbitrary checkpoint. If we could achieve not emitting
> records in notifyCheckpointComplete, stop-with-savepoint --drain could
> be done with one savepoint, and for the normal exit, the operator would
not
> need to wait for other slow operators to exit.
>
> Best,
> Yun
>
>
>
> ------------------Original Mail ------------------
> *Sender:*Kezhu Wang <ke...@gmail.com>
> *Send Date:*Thu Feb 25 15:11:53 2021
> *Recipients:*Flink Dev <de...@flink.apache.org>, Piotr Nowojski <
> piotr.nowojski@gmail.com>
> *CC:*Guowei Ma <gu...@gmail.com>, jingsonglee0@gmail.com <
> jingsonglee0@gmail.com>
> *Subject:*Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> Finished
>
> > Hi all, thanks for driving this and especially Piotr for re-active this
> >
> > thread.
> >
> >
> >
> > First, for `notifyCheckpointComplete`, I have strong preference towards
> >
> > "shut down the dataflow
> >
> > pipeline with one checkpoint in total", so I tend to option dropping
> "send
> >
> > records" from
> >
> > `notifyCheckpointComplete` for next level committing in pipeline, if we
> >
> > ever support it. Without this,
> >
> > we are unable to stop a pipeline manually with all results
materialized.
> >
> >
> >
> > Second, for shutdown unification of `stop-with-savepoint --drain` and
> >
> > FLIP-147, I draft following
> >
> > phase based on emerging proposals(mainly by Piotr and Till) in this
> thread
> >
> > and FLINK-21133.
> >
> >
> >
> > ```java
> >
> > // StreamTask termination phase
> >
> > finish()(Call StreamOperator.finish in chaining order)
> >
> > advanceToEndOfEventTime()(nop if for no-source inputs, this could also
be
> >
> > done in input processor or finish ?)
> >
> > if (there-is-a-pending-terminate-savepoint) {
> >
> > triggerBarrierForDownStream();
> >
> > waitCheckpointComplete();
> >
> > } else if (require-two-phase-commit-in-shutdown) {
> >
> > waitFinalCheckpoint();
> >
> > waitCheckpointComplete();
> >
> > }
> >
> > // TODO: Ignore/Cancel all checkpoints from this point(or hit to
> >
> > coordinator in final checkpoint response ?).
> >
> > close();
> >
> > ```
> >
> >
> >
> > Branches in `if` could reside in different code paths, but the effect
> >
> > should be similar to above.
> >
> > The divergence reflects fact that "stop-with-savepoint --drain" is
> >
> > triggered already while we need
> >
> > to wait for final checkpoint in natural input exhausting.
> >
> >
> >
> > With carefully chosen default functions, we will not break existing
> >
> > interfaces.
> >
> > * `StreamOperator.finish`
> >
> > * `RichFunction.finish`
> >
> >
> >
> > For existing codes:
> >
> > 1. "stop-with-savepoint" with no migration, it should behaves as
before.
> >
> > 2. "require-two-phase-commit-in-shutdown" evaluates to false and no
> >
> > migration, it is same as before.
> >
> > 3. "require-two-phase-commit-in-shutdown" evaluates to true and no
> >
> > migration, records in `close` will
> >
> > lose.
> >
> >
> >
> > For case#3:
> >
> > A. No external materialization in `StreamOperator.close`: Nothing to
> lose.
> >
> > B. External materialization only in `StreamOperator.close`: I don't
think
> >
> > Flink ever claimed that there is
> >
> > any guarantee for this situation.
> >
> > C. External materialization in `notifyCheckpointComplete` from
> >
> > `StreamOperator.close`: But due to fact that
> >
> > `notifyCheckpointComplete` was ignored after operator
> >
> > closed(FLINK-16383), so there will be no external
> >
> > materialization to lose.
> >
> >
> >
> > Then, we could recommend users to migrate possible
> “end-of-stream-flushing”
> >
> > from “close" to “finish”.
> >
> >
> >
> > Best,
> >
> > Kezhu Wang
> >
> >
> >
> >
> >
> > On February 24, 2021 at 23:47:36, Piotr Nowojski (
> piotr.nowojski@gmail.com
> > )
> >
> > wrote:
> >
> >
> >
> > Thanks for the reponses Guowei and Yun,
> >
> >
> >
> > Could you elaborate more/remind me, what does it mean to replace
emitting
> >
> > results from the `notifyCheckpointComplete` with `OperatorCoordinator`
> >
> > approach?
> >
> >
> >
> > About the discussion in FLINK-21133 and how it relates to FLIP-147. You
> are
> >
> > right Yun gao, that in case of `stop-with-savepoint --drain` the whole
> job
> >
> > finishes, while in FLIP-147 some of the sources can work for
arbitrarily
> >
> > long time after others have already finished. However from the runtime
> >
> > perspective, especially how to close a Task and it's operators, there
is
> no
> >
> > difference between those two cases. In both cases, we need to end
input,
> >
> > shut down processing time timers and commit external side effects (two
> >
> > phase commit support) before exiting the task. That's how the
discussion
> >
> > about the behaviour of "stop-with-savepoint" was connected with
FLIP-147.
> >
> >
> >
> > Currently on master, "stop-with-savepoint --drain" drains/flushes
> buffered
> >
> > records and deals correctly with timers, but all of that happens AFTER
> >
> > savepoint was completed. So any records flushed from the operators
during
> >
> > endOfInput/close/shutting down processing timers are never committed to
> >
> > external systems. This is exactly the same problem as the "two phase
> >
> > commit" problem of FLIP-147, that should have the same solution and it
> >
> > should be solved at the same time. For example if we go with the *very
> >
> > undesirable* "one closed operator per one completed
> checkpoint/savepoint",
> >
> > in both cases CheckpointCoordinator, Scheduler and Task would need to
> keep
> >
> > the task alive and keep triggering checkpoints for that task, until all
> >
> > operators in the operator chain are closed (one closed operator per one
> >
> > completed checkpoint).
> >
> >
> >
> > Piotrek
> >
> >
> >
> >
> >
> >
> >
> > śr., 24 lut 2021 o 11:30 Yun Gao napisał(a):
> >
> >
> >
> > > Hi Till, Guowei,
> >
> > >
> >
> > > Very thanks for initiating the disucssion and the deep thoughts!
> >
> > >
> >
> > > For the notifyCheckpointComplete, I also agree we could try to avoid
> >
> > > emitting new records in notifyCheckpointComplete via using
> >
> > > OperatorCoordinator
> >
> > > for new sink API. Besides, the hive sink might also need some
> >
> > modification
> >
> > > for it also emits records in notifyCheckpointComplete.
> >
> > >
> >
> > > For unifying the process of stopping with savepoint and finished due
to
> >
> > > all records
> >
> > > are processed, I also agree with that unifying would always be better
> if
> >
> > > we could achieve,
> >
> > > but I'm still not fully catch up with the implementation: Based on
the
> >
> > > discussion in FLINK-21133,
> >
> > > my understanding is that for stopping with savepoint, now we want to
> >
> > first
> >
> > > stop the source, then we
> >
> > > trigger a savepoint, and after the source received
> >
> > > notifyCheckpointComplete, the source would
> >
> > > start emitting EndOfPartitionEvent to finish the job, am I correct ?
> >
> > >
> >
> > > For normal finish, a difference to me might be if we have multiple
> >
> > > sources, we could not guarantee
> >
> > > when the sources are to finish. We might have one source run one 1
> minute
> >
> > > and another one run for
> >
> > > 1 hour. To unify with the process with stop with savepoint, we might
> need
> >
> > > to hold the fast source until
> >
> > > all the sources are finished? An coordinator would be introduced to
> count
> >
> > > the number of sources
> >
> > > runing and trigger the final savepoint / checkpoint. For the extreme
> >
> > > cases, if we have both bounded and
> >
> > > unbounded sources, we might only count how much bounded source are
> >
> > > remaining ? And if all the bounded
> >
> > > sources are finished we would trigger the special checkpoint. After
all
> >
> > > the bounded part of the graph are
> >
> > > finished, the the remaining part could still do checkpoint and commit
> >
> > data
> >
> > > with FLIP-147.
> >
> > >
> >
> > > Best,
> >
> > > Yun
> >
> > >
> >
> > >
> >
> > >
> >
> > >
> >
> > > ------------------Original Mail ------------------
> >
> > > Sender:Guowei Ma
> >
> > > Send Date:Wed Feb 24 17:35:36 2021
> >
> > > Recipients:dev
> >
> > > CC:Arvid Heise
> >
> > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> Finished
> >
> > > Hi, Till
> >
> > >
> >
> > > Thank you very much for your careful consideration
> >
> > >
> >
> > >
> >
> > >
> >
> > > *1. Emit records in `NotifyCheckpointComplete`.*
> >
> > >
> >
> > > Sorry for making you misunderstanding because of my expression. I
just
> >
> > >
> >
> > > want to say the current interface does not prevent users from doing
it.
> >
> > >
> >
> > > From the perspective of the new sink api, we might not depend on
> emitting
> >
> > >
> >
> > > records in `NotifyCheckpointComplete`, like using
`OperatorCoordinator`
> >
> > >
> >
> > > instead.
> >
> > >
> >
> > >
> >
> > >
> >
> > >
> >
> > >
> >
> > > *2. What does the FLIP-147 guarantee?*I think initially this FLIP
want
> to
> >
> > >
> >
> > > achieve two targets:
> >
> > >
> >
> > > 1. Tasks/Operators exit correctly (as you mentioned the lifecycle of
a
> >
> > >
> >
> > > Task/StreamTask/StreamOperator.).
> >
> > >
> >
> > > 2. Continue to trigger checkpoint after some tasks for mixed jobs.
> >
> > >
> >
> > >
> >
> > >
> >
> > > I think the first thing is related to the discussion in FLINK-21133.
> If I
> >
> > >
> >
> > > understand correctly, in addition to supporting the tasks / operators
> to
> >
> > >
> >
> > > exit correctly, now we also want to unify the process of the tasks
and
> >
> > >
> >
> > > operators for savepoint / finish.
> >
> > >
> >
> > > I think the second thing is orthogonal to the FLINK-21133 because
there
> >
> > are
> >
> > >
> >
> > > topologies that have both the bounded and unbounded input.
> >
> > >
> >
> > >
> >
> > >
> >
> > > *3. How to unify the operator exit process of FLIP-147 with
> >
> > >
> >
> > > stop-with-savepoint?*
> >
> > >
> >
> > > I am not very sure about how to do it yet. But if I understand the
> >
> > >
> >
> > > discussion in the jira correctly it needs to introduce some logic
into
> >
> > >
> >
> > > `CheckpointCoordinator`, which responses for triggering “the unified
> >
> > >
> >
> > > operator exit process”. Am I correct?
> >
> > >
> >
> > >
> >
> > >
> >
> > > Best,
> >
> > >
> >
> > > Guowei
> >
> > >
> >
> > >
> >
> > >
> >
> > >
> >
> > >
> >
> > > On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann wrote:
> >
> > >
> >
> > >
> >
> > >
> >
> > > > Thanks for the explanation Yun and Guowei. I have to admit that I
do
> >
> > not
> >
> > >
> >
> > > > fully understand why this is strictly required but I think that we
> are
> >
> > >
> >
> > > > touching two very important aspects which might have far fetching
> >
> > >
> >
> > > > consequences for how Flink works:
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > 1) Do we want to allow that multiple checkpoints are required to
> >
> > >
> >
> > > > materialize results?
> >
> > >
> >
> > > > 2) Do we want to allow to emit records in notifyCheckpointComplete?
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > For 1) I am not sure whether this has been discussed within the
> >
> > community
> >
> > >
> >
> > > > sufficiently. Requiring multiple checkpoints to materialize a
result
> >
> > >
> >
> > > > because of multi level committers has the consequence that we
> increase
> >
> > > the
> >
> > >
> >
> > > > latency from checkpoint interval to #levels * checkpoint interval.
> >
> > >
> >
> > > > Moreover, having to drain the pipeline in multiple steps, would
break
> >
> > the
> >
> > >
> >
> > > > stop-with-savepoint --drain because which savepoint do you report
to
> >
> > the
> >
> > >
> >
> > > > user?
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > For 2) allowing to send records after the final
> >
> > notifyCheckpointComplete
> >
> > >
> >
> > > > will effectively mean that we need to shut down a topology in
> multiple
> >
> > >
> >
> > > > steps (in the worst case one operator per checkpoint). This would
be
> a
> >
> > >
> >
> > > > strong argument for not allowing this to me. The fact that users
can
> >
> > send
> >
> > >
> >
> > > > records after the notifyCheckpointComplete is more by accident than
> by
> >
> > >
> >
> > > > design. I think we should make this a very deliberate decision and
in
> >
> > > doubt
> >
> > >
> >
> > > > I would be in favour of a more restrictive model unless there is a
> very
> >
> > >
> >
> > > > good reason why this should be supported.
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > Taking also the discussion in FLINK-21133 [1] into account, it
seems
> to
> >
> > > me
> >
> > >
> >
> > > > that we haven't really understood what kind of guarantees we want
to
> >
> > give
> >
> > >
> >
> > > > to our users and how the final checkpoint should exactly work. I
> >
> > > understand
> >
> > >
> >
> > > > that this is not included in the first scope of FLIP-147 but I
think
> >
> > this
> >
> > >
> >
> > > > is so important that we should figure this out asap. Also because
the
> >
> > > exact
> >
> > >
> >
> > > > shut down behaviour will have to be aligned with the lifecycle of a
> >
> > >
> >
> > > > Task/StreamTask/StreamOperator. And last but not least because
other
> >
> > >
> >
> > > > features such as the new sink API start building upon a shut down
> model
> >
> > >
> >
> > > > which has not been fully understood/agreed upon.
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > [1] https://issues.apache.org/jira/browse/FLINK-21133
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > Cheers,
> >
> > >
> >
> > > > Till
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma wrote:
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > > Thanks Yun for the detailed explanation.
> >
> > >
> >
> > > > > A simple supplementary explanation about the sink case: Maybe we
> >
> > could
> >
> > >
> >
> > > > use
> >
> > >
> >
> > > > > `OperatorCoordinator` to avoid sending the element to the
> downstream
> >
> > >
> >
> > > > > operator.
> >
> > >
> >
> > > > > But I agree we could not limit the users not to emit records in
the
> >
> > >
> >
> > > > > `notiyCheckpointComplete`.
> >
> > >
> >
> > > > >
> >
> > >
> >
> > > > > Best,
> >
> > >
> >
> > > > > Guowei
> >
> > >
> >
> > > > >
> >
> > >
> >
> > > > >
> >
> > >
> >
> > > > > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao
> >
> > >
> >
> > > > > wrote:
> >
> > >
> >
> > > > >
> >
> > >
> >
> > > > > > Hi all,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > I'd like to first detail the issue with emitting records in
> >
> > >
> >
> > > > > > notifyCheckpointComplete for context. For specific usage,
> >
> > >
> >
> > > > > > an example would be for sink, it might want to write some
> metadata
> >
> > >
> >
> > > > after
> >
> > >
> >
> > > > > > all the transactions are committed
> >
> > >
> >
> > > > > > (like write a marker file _SUCCESS to the output directory).
This
> >
> > > case
> >
> > >
> >
> > > > is
> >
> > >
> >
> > > > > > currently supported via the two level
> >
> > >
> >
> > > > > > committers of the new sink API: when received endOfInput(), the
> >
> > >
> >
> > > > Committer
> >
> > >
> >
> > > > > > wait for another checkpoint to
> >
> > >
> >
> > > > > > commits all the pending transactions and emit the list of files
> to
> >
> > > the
> >
> > >
> >
> > > > > > GlobalCommitter. The GlobalCommitter
> >
> > >
> >
> > > > > > would wait for another checkpoint to also write the metadata
with
> >
> > 2pc
> >
> > >
> >
> > > > > > (Although sometimes 2pc is not needed
> >
> > >
> >
> > > > > > for writing metadata, it should be only an optimization and
still
> >
> > >
> >
> > > > > requires
> >
> > >
> >
> > > > > > the Committer do commit before
> >
> > >
> >
> > > > > > notifying the global Committer. Also another note is
> >
> > GlobalCommitter
> >
> > > is
> >
> > >
> >
> > > > > > also added for some other cases
> >
> > >
> >
> > > > > > like some sinks want an commiter with dop = 1, like
IceBergSink).
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > However, a more general issue to me is that currently we do not
> >
> > limit
> >
> > >
> >
> > > > > > users to not emit records in
> >
> > >
> >
> > > > > > notifyCheckpointComplete in the API level. The sink case could
be
> >
> > >
> >
> > > > viewed
> >
> > >
> >
> > > > > > as a special case, but in addition
> >
> > >
> >
> > > > > > to this one, logically users could also implement their own
cases
> >
> > > that
> >
> > >
> >
> > > > > > emits records in notifyCheckpointComplete.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > Best,
> >
> > >
> >
> > > > > > Yun
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > ------------------Original Mail ------------------
> >
> > >
> >
> > > > > > Sender:Arvid Heise
> >
> > >
> >
> > > > > > Send Date:Fri Feb 12 20:46:04 2021
> >
> > >
> >
> > > > > > Recipients:dev
> >
> > >
> >
> > > > > > CC:Yun Gao
> >
> > >
> >
> > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> >
> > >
> >
> > > > Finished
> >
> > >
> >
> > > > > > Hi Piotr,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > Thank you for raising your concern. Unfortunately, I do not
have
> a
> >
> > >
> >
> > > > better
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > idea than doing closing of operators intermittently with
> >
> > checkpoints
> >
> > > (=
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > multiple last checkpoints).
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > However, two ideas on how to improve the overall user
experience:
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > 1. If an operator is not relying on notifyCheckpointComplete,
we
> >
> > can
> >
> > >
> >
> > > > > close
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > it faster (without waiting for a checkpoint). In general, I'd
> >
> > assume
> >
> > >
> >
> > > > that
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > almost all non-sinks behave that way.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > 2. We may increase the checkpointing frequency for the last
> >
> > >
> >
> > > > checkpoints.
> >
> > >
> >
> > > > > We
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > need to avoid overloading checkpoint storages and task
managers,
> >
> > but
> >
> > > I
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > assume the more operators are closed, the lower the
checkpointing
> >
> > >
> >
> > > > > interval
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > can be.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > For 1, I'd propose to add (name TBD):
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > default boolean StreamOperator#requiresFinalCheckpoint() {
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > return true;
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > }
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > This means all operators are conservatively (=slowly) closed.
For
> >
> > > most
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > operators, we can then define their behavior by overriding in
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > AbstractUdfStreamOperator
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > @Override
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > return userFunction instanceof CheckpointListener;
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > }
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > This idea can be further refined in also adding
> >
> > > requiresFinalCheckpoint
> >
> > >
> >
> > > > > to
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > CheckpointListener to exclude all operators with UDFs that
> >
> > implement
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > CheckpointListener but do not need it for 2pc.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > @Override
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > return userFunction instanceof CheckpointListener &&
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > ((CheckpointListener) userFunction).requiresFinalCheckpoint();
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > }
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > That approach would also work for statebackends/snapshot
> strategies
> >
> > >
> >
> > > > that
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > require some 2pc.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > If we can contain it to the @PublicEvolving StreamOperator, it
> >
> > would
> >
> > > be
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > better of course.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > Best,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > Arvid
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > wrote:
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > Hey,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > I would like to raise a concern about implementation of the
> final
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > checkpoints taking into account operators/functions that are
> >
> > >
> >
> > > > > implementing
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > two phase commit (2pc) protocol for exactly-once processing
> with
> >
> > > some
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > external state (kept outside of the Flink). Primarily
> >
> > exactly-once
> >
> > >
> >
> > > > > sinks.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > First of all, as I understand it, this is not planned in the
> >
> > first
> >
> > >
> >
> > > > > > version
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > of this FLIP. I'm fine with that, however I would strongly
> >
> > > emphasize
> >
> > >
> >
> > > > > this
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > in every place we will be mentioning FLIP-147 efforts. This
is
> >
> > >
> >
> > > > because
> >
> > >
> >
> > > > > > me,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > as a user, upon hearing "Flink supports checkpointing with
> >
> > bounded
> >
> > >
> >
> > > > > > inputs"
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > I would expect 2pc to work properly and to commit the
external
> >
> > side
> >
> > >
> >
> > > > > > effects
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > upon finishing. As it is now, I (as a user) would be
surprised
> >
> > > with a
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > silent data loss (of not committed trailing data). This is
> just a
> >
> > >
> >
> > > > > remark,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > that we need to attach this warning to every blog
> >
> > >
> >
> > > > > post/documentation/user
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > mailing list response related to "Support Checkpoints After
> Tasks
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > Finished". Also I would suggest to prioritize the follow up
of
> >
> > >
> >
> > > > > supporting
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 2pc.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > Secondly, I think we are missing how difficult and
problematic
> >
> > will
> >
> > >
> >
> > > > be
> >
> > >
> >
> > > > > > 2pc
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > support with the final checkpoint.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > For starters, keep in mind that currently 2pc can be
> implemented
> >
> > by
> >
> > >
> >
> > > > > users
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > using both `@Public` APIs as functions and `@PublicEvolving`
> >
> > >
> >
> > > > operators
> >
> > >
> >
> > > > > in
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > any place in the job graph. It's not limited to only the
sinks.
> >
> > For
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > example users could easily implement the `AsynFunction` (for
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > `AsyncWaitOperator`) that is using 2pc based on the
> >
> > >
> >
> > > > > `CheckpointListener`
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > interface. I'm not saying it's common, probably just a tiny
> >
> > > minority
> >
> > >
> >
> > > > of
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > users are doing that (if any at all), but nevertheless that's
> >
> > >
> >
> > > > possible
> >
> > >
> >
> > > > > > and
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > currently (implicitly?) supported in Flink.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > Next complication is the support of bounded streams
> >
> > >
> >
> > > > (`BoundedOneInput`
> >
> > >
> >
> > > > > or
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > `BoundedMultiInput` interfaces) and the closing/shutdown
> >
> > procedure
> >
> > > of
> >
> > >
> >
> > > > > the
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > operators. Currently it works as follows:
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 1. `endOfInput` is called on the first operator in the chain
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 2. We quiesce the processing timers
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> for
> >
> > > the
> >
> > >
> >
> > > > > > first
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > operator, so no new timers will be triggered
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 3. We wait for the already fired timers to finish executing
> >
> > > (spinning
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > mailbox loop)
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 4. We are closing the first operator
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 5. We go to the next (second) operator in the chain and
repeat
> >
> > the
> >
> > >
> >
> > > > > steps
> >
> > >
> >
> > > > > > 1.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > to 5.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > This is because operators can emit data after processing
> >
> > >
> >
> > > > `endOfInput`,
> >
> > >
> >
> > > > > > from
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > timers, async mailbox actions and inside the `close` method
> >
> > itself.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > Now the problem is to support the final checkpoint with 2pc,
we
> >
> > > need
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > trigger `snapshotState` and `notifyCheckpointComplete` call
at
> >
> > the
> >
> > >
> >
> > > > very
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > least only after `endOfInput` call on the operator. Probably
> the
> >
> > > best
> >
> > >
> >
> > > > > > place
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > would be in between steps 3. and 4. However that means, we
> would
> >
> > be
> >
> > >
> >
> > > > > > forced
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > to wait for steps 1. to 3. to finish, then wait for a next
> >
> > > checkpoint
> >
> > >
> >
> > > > > to
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > trigger AND complete, before finally closing the head
operator,
> >
> > and
> >
> > >
> >
> > > > > only
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > then we can start closing the next operator in the chain:
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 1. `endOfInput` is called on the first operator in the chain
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 2. We quiesce the processing timers
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> for
> >
> > > the
> >
> > >
> >
> > > > > > first
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > operator, so no new timers will be triggered
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 3. We wait for the already fired timers to finish executing
> >
> > > (spinning
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > mailbox loop)
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > *3b. We wait for one more checkpoint to trigger and for the
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > `notifyCheckpointComplete` RPC.*
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 4. We are closing the first operator
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 5. We go to the next (second) operator in the chain and
repeat
> >
> > the
> >
> > >
> >
> > > > > steps
> >
> > >
> >
> > > > > > 1.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > to 5.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > That means, we can close one operator per successful
> checkpoint.
> >
> > To
> >
> > >
> >
> > > > > close
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 10 operators, we would need 10 successful checkpoints.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > I was thinking about different approaches to this problem,
and
> I
> >
> > >
> >
> > > > > couldn't
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > find any viable ones. All I could think of would break the
> >
> > current
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > `@Public` API and/or would be ugly/confusing for the users.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > For example a relatively simple solution, to introduce a
> >
> > `preClose`
> >
> > >
> >
> > > > or
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > `flush` method to the operators, with a contract that after
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > `flush`, operators would be forbidden from emitting more
> records,
> >
> > > so
> >
> > >
> >
> > > > > that
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > we can replace step 4. with this `flush` call, and then
having
> a
> >
> > >
> >
> > > > single
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > checkpoint to finish 2pc for all of the operators inside the
> >
> > chain,
> >
> > >
> >
> > > > > > doesn't
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > work. Sheer fact of adding this `flush` method and changing
the
> >
> > >
> >
> > > > > contract
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > would break the current API and Yun Gao has pointed out to
me,
> >
> > that
> >
> > >
> >
> > > > we
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > either already support, or want to support operators that are
> >
> > >
> >
> > > > emitting
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > records from within the `notifyCheckpointComplete` call:
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > Yun Gao:
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > like with the new sink api there might be writer ->
committer
> >
> > ->
> >
> > >
> >
> > > > > global
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > committer, the committer would need to wait for the last
> >
> > checkpoint
> >
> > >
> >
> > > > to
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > commit
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > the last piece of data, and after that it also need to emit
> the
> >
> > >
> >
> > > > list
> >
> > >
> >
> > > > > of
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > transactions get committed to global committer to do some
> >
> > >
> >
> > > > finalization
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > logic.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > So it wouldn't solve the problem (at least not fully).
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > I don't know if anyone has any better ideas how to solve this
> >
> > >
> >
> > > > problem?
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > Piotrek
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > pt., 15 sty 2021 o 14:57 Yun Gao
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > napisał(a):
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > Hi Aljoscha,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > I think so since we seems to do not have other divergence
and
> >
> > new
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > objections now. I'll open the vote then. Very thanks!
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > Best,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > Yun
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > > ------------------------------------------------------------------
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > From:Aljoscha Krettek
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > Send Time:2021 Jan. 15 (Fri.) 21:24
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > To:dev
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After
> Tasks
> >
> > >
> >
> > > > > > Finished
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > Thanks for the summary! I think we can now move towards a
> >
> > [VOTE]
> >
> > >
> >
> > > > > > thread,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > right?
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > On 2021/01/15 13:43, Yun Gao wrote:
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > >1) For the problem that the "new" root task coincidently
> >
> > > finished
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > >before getting triggered successfully, we have listed two
> >
> > > options
> >
> > >
> >
> > > > in
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > >the FLIP-147[1], for the first version, now we are not
tend
> to
> >
> > > go
> >
> > >
> >
> > > > > with
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > >the first option that JM would re-compute and re-trigger
new
> >
> > >
> >
> > > > sources
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > >when it realized some tasks are not triggered
successfully.
> >
> > This
> >
> > >
> >
> > > > > > option
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > >would avoid the complexity of adding new PRC and
duplicating
> >
> > > task
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > >states, and in average case it would not cause too much
> >
> > > overhead.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > You wrote "we are *not* tend to go with the first option",
> but
> >
> > I
> >
> > >
> >
> > > > > think
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > you meant wo write "we tend to *now* go with the first
> option",
> >
> > >
> >
> > > > > right?
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > That's also how it is in the FLIP, I just wanted to clarify
> for
> >
> > > the
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > mailing list.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > >
> >
> > >
> >
> > > >
> >
> > >
> >
> > >
> >
> >
>

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

Posted by Till Rohrmann <tr...@apache.org>.
Thanks for all your thoughts. I think we should further think through
whether to allow checkpoints after an operator has emitted all its records
(e.g. after close is called currently) or not. I think by doing this we
would nicely decouple the checkpoint taking from the operator lifecycle and
wouldn't need special checkpoints/savepoints for the final checkpoint and
stop-with-savepoint --drain. Let me try to explain this a bit more detailed.

If we say an operator has the rough lifecycle RUNNING => TERMINATING =>
TERMINATED where we go from RUNNING into TERMINATING after we have seen the
EndOfPartitionEvent and flushed all our records. The operator goes from
TERMINATING => TERMINATED if it has persisted all its possible side
effects. Throughout all states, it is possible to trigger a checkpoint. A
stateless operator will immediately go from TERMINATING to TERMINATED
whereas a stateful operator would wait for another checkpoint to be
triggered and successfully completed (notifyCheckpointComplete).

With this model, the final checkpoint is quite simple because it is
ingrained in the lifecycle of an operator. Differently said an operator
will only terminate after it has committed its side effects and seen the
notifyCheckpointComplete message (if it is stateful). Here it is important
to note that in the streaming case, different bounded operators can
terminate at different times. They don't have to terminate all with the
same checkpoint.

How to implement the stop-with-savepoint --drain/terminate command with
this model: One idea could be to tell the sources that they should stop
reading. This should trigger the EndOfPartitionEvent to be sent downstream.
This will transition all operators into the TERMINATING state. Next the JM
can trigger a checkpoint to shut the operators down and not to wait for the
next automatic checkpoint trigger event.

By allowing checkpoints throughout the entire lifecycle of an operator we
disallow sending records from notifyCheckpointComplete because this message
will also be sent in the state TERMINATING where an operator has already
produced all of its records.

What do you think? Maybe this model overlooks some important cases. One
downside is that we will break the operator API with changing the lifecycle
of an operator.

Cheers,
Till



On Fri, Feb 26, 2021 at 7:29 AM Kezhu Wang <ke...@gmail.com> wrote:

> Hi Yun,
>
> The termination phase I drafted depends on a de-fact that
> `notifyCheckpointComplete` is ignored currently after `close`. Thus, any
> external materialization inside `close`(including `endInput`) or
> `notifyCheckpointComplete` is either not guaranteed or not committed.
>
> I tried to emphasize no-breaking changes and post-pone migration in later
> releases for this termination phase. But now, I am kind of worry about
> whether it will cause long-term maintenance hell.
>
> Personally, I think allowing checkpoint after `close`(@Till proposed this
> in FLINK-21133) could be the minimal change. But there are concerns:
> * It will break some existing code possibly in a silent way.(@Piotr pointed
> this already in FLINK-21133)
> * I think the name `close` is kind of misleading. (@Piotr suggested
> renaming this to `finish` to not break code silently in FLINK-21133)
>
> > stop-with-savepoint --drain would wait for a specific savepoint
>
> For stop-with-savepoint, the checkpoint barrier is already created or
> received there.
>
>
> Best,
> Kezhu Wang
>
> On February 25, 2021 at 21:02:30, Yun Gao (yungao.gy@aliyun.com) wrote:
>
> Hi all,
>
> Very thanks for the discussions!
>
>
>
> A. Regarding how to avoid emitting records in notifyCheckpointComplete:
>
> Currently the structure of a new sink is writer -> committer -> global
> committer and the paralellism of global committer
> must be one. By design it would be used in several cases:
> 1. writer -> committer: for normal sinks that write pending data in writer
> and commit pending data in committer.
> 2. writer -> global committer: for sinks require the committer's
> parallelism be 1, like iceberg sink which
> relies on optimistic lock to commit data hopes to reduce the conflicts.
> 3. writer -> committer -> global committer: for sinks that also need to
> write meta data (like _SUCCESS file
> or add record in hive metastore) in global committer.
>
> The third case would cause the cascade commit problem. To overcome this
> problem, we would like to
> 1. Previously the global committer also support writing metadata with 2pc.
> Now we disable this mode
> and always rely on the property that writing metadata is repeatable. With
> this limitation there should no
> need of cascade commit, but the committer would still emit records in
> notifyCheckpointComplete.
> 2. We further move global committer in the case 3 to the operator
> coordinator. Then the committer would
> not need to emit records, but communicates with its operator coordinator.
>
> One core issue for using the OperatorCoordinator is how to keep the
> communication between the operator
> and the OperatorCoordinator exactly-once. Since the message is always from
> the oeprator to the OperatorCoordinator
> in this case, we would only need to bookkeep the message sent between the
> OperatorCoordinator takes snapshot and
> the Operator takes snapshot in the state of the Operator.
>
> On how to achieve the change in detail we would still need some think, it
> currently seems we would have to had
> some modification to the current new sink api.
>
>
>
> B. Regarding the stop-with-savepoint --drain
>
> Very thanks @Piotr for the further explanation and now I realize I have
> understand wrongly for the semantics of
> stop-with-savepoint --drain. Now I think that the problem should be we
> should also include the records produced in
> `endOfInput()` and `close()` also in the last savepoint, am I correct? If
> so, it seems we still have some undetermined options for
> the lifecycle of the operator, like in Kezhu's proposal the close() happens
> at last, but it seems close() might also emit records (
> so now the operator are closed with op1's close() -> op2's endOfInput() ->
> op2's close() -> op3's endOfinput -> ...) ?
>
> And on the other side, as Kezhu has also proposed, perhapse we might have
> the stop-with-savepoint --drain and normal exit in the same process,
> but have slightly difference in that stop-with-savepoint --drain would wait
> for a specific savepoint and in normal exit, the operator
> might wait for arbitrary checkpoint. If we could achieve not emitting
> records in notifyCheckpointComplete, stop-with-savepoint --drain could
> be done with one savepoint, and for the normal exit, the operator would not
> need to wait for other slow operators to exit.
>
> Best,
> Yun
>
>
>
> ------------------Original Mail ------------------
> *Sender:*Kezhu Wang <ke...@gmail.com>
> *Send Date:*Thu Feb 25 15:11:53 2021
> *Recipients:*Flink Dev <de...@flink.apache.org>, Piotr Nowojski <
> piotr.nowojski@gmail.com>
> *CC:*Guowei Ma <gu...@gmail.com>, jingsonglee0@gmail.com <
> jingsonglee0@gmail.com>
> *Subject:*Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> Finished
>
> > Hi all, thanks for driving this and especially Piotr for re-active this
> >
> > thread.
> >
> >
> >
> > First, for `notifyCheckpointComplete`, I have strong preference towards
> >
> > "shut down the dataflow
> >
> > pipeline with one checkpoint in total", so I tend to option dropping
> "send
> >
> > records" from
> >
> > `notifyCheckpointComplete` for next level committing in pipeline, if we
> >
> > ever support it. Without this,
> >
> > we are unable to stop a pipeline manually with all results materialized.
> >
> >
> >
> > Second, for shutdown unification of `stop-with-savepoint --drain` and
> >
> > FLIP-147, I draft following
> >
> > phase based on emerging proposals(mainly by Piotr and Till) in this
> thread
> >
> > and FLINK-21133.
> >
> >
> >
> > ```java
> >
> > // StreamTask termination phase
> >
> > finish()(Call StreamOperator.finish in chaining order)
> >
> > advanceToEndOfEventTime()(nop if for no-source inputs, this could also be
> >
> > done in input processor or finish ?)
> >
> > if (there-is-a-pending-terminate-savepoint) {
> >
> > triggerBarrierForDownStream();
> >
> > waitCheckpointComplete();
> >
> > } else if (require-two-phase-commit-in-shutdown) {
> >
> > waitFinalCheckpoint();
> >
> > waitCheckpointComplete();
> >
> > }
> >
> > // TODO: Ignore/Cancel all checkpoints from this point(or hit to
> >
> > coordinator in final checkpoint response ?).
> >
> > close();
> >
> > ```
> >
> >
> >
> > Branches in `if` could reside in different code paths, but the effect
> >
> > should be similar to above.
> >
> > The divergence reflects fact that "stop-with-savepoint --drain" is
> >
> > triggered already while we need
> >
> > to wait for final checkpoint in natural input exhausting.
> >
> >
> >
> > With carefully chosen default functions, we will not break existing
> >
> > interfaces.
> >
> > * `StreamOperator.finish`
> >
> > * `RichFunction.finish`
> >
> >
> >
> > For existing codes:
> >
> > 1. "stop-with-savepoint" with no migration, it should behaves as before.
> >
> > 2. "require-two-phase-commit-in-shutdown" evaluates to false and no
> >
> > migration, it is same as before.
> >
> > 3. "require-two-phase-commit-in-shutdown" evaluates to true and no
> >
> > migration, records in `close` will
> >
> > lose.
> >
> >
> >
> > For case#3:
> >
> > A. No external materialization in `StreamOperator.close`: Nothing to
> lose.
> >
> > B. External materialization only in `StreamOperator.close`: I don't think
> >
> > Flink ever claimed that there is
> >
> > any guarantee for this situation.
> >
> > C. External materialization in `notifyCheckpointComplete` from
> >
> > `StreamOperator.close`: But due to fact that
> >
> > `notifyCheckpointComplete` was ignored after operator
> >
> > closed(FLINK-16383), so there will be no external
> >
> > materialization to lose.
> >
> >
> >
> > Then, we could recommend users to migrate possible
> “end-of-stream-flushing”
> >
> > from “close" to “finish”.
> >
> >
> >
> > Best,
> >
> > Kezhu Wang
> >
> >
> >
> >
> >
> > On February 24, 2021 at 23:47:36, Piotr Nowojski (
> piotr.nowojski@gmail.com
> > )
> >
> > wrote:
> >
> >
> >
> > Thanks for the reponses Guowei and Yun,
> >
> >
> >
> > Could you elaborate more/remind me, what does it mean to replace emitting
> >
> > results from the `notifyCheckpointComplete` with `OperatorCoordinator`
> >
> > approach?
> >
> >
> >
> > About the discussion in FLINK-21133 and how it relates to FLIP-147. You
> are
> >
> > right Yun gao, that in case of `stop-with-savepoint --drain` the whole
> job
> >
> > finishes, while in FLIP-147 some of the sources can work for arbitrarily
> >
> > long time after others have already finished. However from the runtime
> >
> > perspective, especially how to close a Task and it's operators, there is
> no
> >
> > difference between those two cases. In both cases, we need to end input,
> >
> > shut down processing time timers and commit external side effects (two
> >
> > phase commit support) before exiting the task. That's how the discussion
> >
> > about the behaviour of "stop-with-savepoint" was connected with FLIP-147.
> >
> >
> >
> > Currently on master, "stop-with-savepoint --drain" drains/flushes
> buffered
> >
> > records and deals correctly with timers, but all of that happens AFTER
> >
> > savepoint was completed. So any records flushed from the operators during
> >
> > endOfInput/close/shutting down processing timers are never committed to
> >
> > external systems. This is exactly the same problem as the "two phase
> >
> > commit" problem of FLIP-147, that should have the same solution and it
> >
> > should be solved at the same time. For example if we go with the *very
> >
> > undesirable* "one closed operator per one completed
> checkpoint/savepoint",
> >
> > in both cases CheckpointCoordinator, Scheduler and Task would need to
> keep
> >
> > the task alive and keep triggering checkpoints for that task, until all
> >
> > operators in the operator chain are closed (one closed operator per one
> >
> > completed checkpoint).
> >
> >
> >
> > Piotrek
> >
> >
> >
> >
> >
> >
> >
> > śr., 24 lut 2021 o 11:30 Yun Gao napisał(a):
> >
> >
> >
> > > Hi Till, Guowei,
> >
> > >
> >
> > > Very thanks for initiating the disucssion and the deep thoughts!
> >
> > >
> >
> > > For the notifyCheckpointComplete, I also agree we could try to avoid
> >
> > > emitting new records in notifyCheckpointComplete via using
> >
> > > OperatorCoordinator
> >
> > > for new sink API. Besides, the hive sink might also need some
> >
> > modification
> >
> > > for it also emits records in notifyCheckpointComplete.
> >
> > >
> >
> > > For unifying the process of stopping with savepoint and finished due to
> >
> > > all records
> >
> > > are processed, I also agree with that unifying would always be better
> if
> >
> > > we could achieve,
> >
> > > but I'm still not fully catch up with the implementation: Based on the
> >
> > > discussion in FLINK-21133,
> >
> > > my understanding is that for stopping with savepoint, now we want to
> >
> > first
> >
> > > stop the source, then we
> >
> > > trigger a savepoint, and after the source received
> >
> > > notifyCheckpointComplete, the source would
> >
> > > start emitting EndOfPartitionEvent to finish the job, am I correct ?
> >
> > >
> >
> > > For normal finish, a difference to me might be if we have multiple
> >
> > > sources, we could not guarantee
> >
> > > when the sources are to finish. We might have one source run one 1
> minute
> >
> > > and another one run for
> >
> > > 1 hour. To unify with the process with stop with savepoint, we might
> need
> >
> > > to hold the fast source until
> >
> > > all the sources are finished? An coordinator would be introduced to
> count
> >
> > > the number of sources
> >
> > > runing and trigger the final savepoint / checkpoint. For the extreme
> >
> > > cases, if we have both bounded and
> >
> > > unbounded sources, we might only count how much bounded source are
> >
> > > remaining ? And if all the bounded
> >
> > > sources are finished we would trigger the special checkpoint. After all
> >
> > > the bounded part of the graph are
> >
> > > finished, the the remaining part could still do checkpoint and commit
> >
> > data
> >
> > > with FLIP-147.
> >
> > >
> >
> > > Best,
> >
> > > Yun
> >
> > >
> >
> > >
> >
> > >
> >
> > >
> >
> > > ------------------Original Mail ------------------
> >
> > > Sender:Guowei Ma
> >
> > > Send Date:Wed Feb 24 17:35:36 2021
> >
> > > Recipients:dev
> >
> > > CC:Arvid Heise
> >
> > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> Finished
> >
> > > Hi, Till
> >
> > >
> >
> > > Thank you very much for your careful consideration
> >
> > >
> >
> > >
> >
> > >
> >
> > > *1. Emit records in `NotifyCheckpointComplete`.*
> >
> > >
> >
> > > Sorry for making you misunderstanding because of my expression. I just
> >
> > >
> >
> > > want to say the current interface does not prevent users from doing it.
> >
> > >
> >
> > > From the perspective of the new sink api, we might not depend on
> emitting
> >
> > >
> >
> > > records in `NotifyCheckpointComplete`, like using `OperatorCoordinator`
> >
> > >
> >
> > > instead.
> >
> > >
> >
> > >
> >
> > >
> >
> > >
> >
> > >
> >
> > > *2. What does the FLIP-147 guarantee?*I think initially this FLIP want
> to
> >
> > >
> >
> > > achieve two targets:
> >
> > >
> >
> > > 1. Tasks/Operators exit correctly (as you mentioned the lifecycle of a
> >
> > >
> >
> > > Task/StreamTask/StreamOperator.).
> >
> > >
> >
> > > 2. Continue to trigger checkpoint after some tasks for mixed jobs.
> >
> > >
> >
> > >
> >
> > >
> >
> > > I think the first thing is related to the discussion in FLINK-21133.
> If I
> >
> > >
> >
> > > understand correctly, in addition to supporting the tasks / operators
> to
> >
> > >
> >
> > > exit correctly, now we also want to unify the process of the tasks and
> >
> > >
> >
> > > operators for savepoint / finish.
> >
> > >
> >
> > > I think the second thing is orthogonal to the FLINK-21133 because there
> >
> > are
> >
> > >
> >
> > > topologies that have both the bounded and unbounded input.
> >
> > >
> >
> > >
> >
> > >
> >
> > > *3. How to unify the operator exit process of FLIP-147 with
> >
> > >
> >
> > > stop-with-savepoint?*
> >
> > >
> >
> > > I am not very sure about how to do it yet. But if I understand the
> >
> > >
> >
> > > discussion in the jira correctly it needs to introduce some logic into
> >
> > >
> >
> > > `CheckpointCoordinator`, which responses for triggering “the unified
> >
> > >
> >
> > > operator exit process”. Am I correct?
> >
> > >
> >
> > >
> >
> > >
> >
> > > Best,
> >
> > >
> >
> > > Guowei
> >
> > >
> >
> > >
> >
> > >
> >
> > >
> >
> > >
> >
> > > On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann wrote:
> >
> > >
> >
> > >
> >
> > >
> >
> > > > Thanks for the explanation Yun and Guowei. I have to admit that I do
> >
> > not
> >
> > >
> >
> > > > fully understand why this is strictly required but I think that we
> are
> >
> > >
> >
> > > > touching two very important aspects which might have far fetching
> >
> > >
> >
> > > > consequences for how Flink works:
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > 1) Do we want to allow that multiple checkpoints are required to
> >
> > >
> >
> > > > materialize results?
> >
> > >
> >
> > > > 2) Do we want to allow to emit records in notifyCheckpointComplete?
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > For 1) I am not sure whether this has been discussed within the
> >
> > community
> >
> > >
> >
> > > > sufficiently. Requiring multiple checkpoints to materialize a result
> >
> > >
> >
> > > > because of multi level committers has the consequence that we
> increase
> >
> > > the
> >
> > >
> >
> > > > latency from checkpoint interval to #levels * checkpoint interval.
> >
> > >
> >
> > > > Moreover, having to drain the pipeline in multiple steps, would break
> >
> > the
> >
> > >
> >
> > > > stop-with-savepoint --drain because which savepoint do you report to
> >
> > the
> >
> > >
> >
> > > > user?
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > For 2) allowing to send records after the final
> >
> > notifyCheckpointComplete
> >
> > >
> >
> > > > will effectively mean that we need to shut down a topology in
> multiple
> >
> > >
> >
> > > > steps (in the worst case one operator per checkpoint). This would be
> a
> >
> > >
> >
> > > > strong argument for not allowing this to me. The fact that users can
> >
> > send
> >
> > >
> >
> > > > records after the notifyCheckpointComplete is more by accident than
> by
> >
> > >
> >
> > > > design. I think we should make this a very deliberate decision and in
> >
> > > doubt
> >
> > >
> >
> > > > I would be in favour of a more restrictive model unless there is a
> very
> >
> > >
> >
> > > > good reason why this should be supported.
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > Taking also the discussion in FLINK-21133 [1] into account, it seems
> to
> >
> > > me
> >
> > >
> >
> > > > that we haven't really understood what kind of guarantees we want to
> >
> > give
> >
> > >
> >
> > > > to our users and how the final checkpoint should exactly work. I
> >
> > > understand
> >
> > >
> >
> > > > that this is not included in the first scope of FLIP-147 but I think
> >
> > this
> >
> > >
> >
> > > > is so important that we should figure this out asap. Also because the
> >
> > > exact
> >
> > >
> >
> > > > shut down behaviour will have to be aligned with the lifecycle of a
> >
> > >
> >
> > > > Task/StreamTask/StreamOperator. And last but not least because other
> >
> > >
> >
> > > > features such as the new sink API start building upon a shut down
> model
> >
> > >
> >
> > > > which has not been fully understood/agreed upon.
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > [1] https://issues.apache.org/jira/browse/FLINK-21133
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > Cheers,
> >
> > >
> >
> > > > Till
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma wrote:
> >
> > >
> >
> > > >
> >
> > >
> >
> > > > > Thanks Yun for the detailed explanation.
> >
> > >
> >
> > > > > A simple supplementary explanation about the sink case: Maybe we
> >
> > could
> >
> > >
> >
> > > > use
> >
> > >
> >
> > > > > `OperatorCoordinator` to avoid sending the element to the
> downstream
> >
> > >
> >
> > > > > operator.
> >
> > >
> >
> > > > > But I agree we could not limit the users not to emit records in the
> >
> > >
> >
> > > > > `notiyCheckpointComplete`.
> >
> > >
> >
> > > > >
> >
> > >
> >
> > > > > Best,
> >
> > >
> >
> > > > > Guowei
> >
> > >
> >
> > > > >
> >
> > >
> >
> > > > >
> >
> > >
> >
> > > > > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao
> >
> > >
> >
> > > > > wrote:
> >
> > >
> >
> > > > >
> >
> > >
> >
> > > > > > Hi all,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > I'd like to first detail the issue with emitting records in
> >
> > >
> >
> > > > > > notifyCheckpointComplete for context. For specific usage,
> >
> > >
> >
> > > > > > an example would be for sink, it might want to write some
> metadata
> >
> > >
> >
> > > > after
> >
> > >
> >
> > > > > > all the transactions are committed
> >
> > >
> >
> > > > > > (like write a marker file _SUCCESS to the output directory). This
> >
> > > case
> >
> > >
> >
> > > > is
> >
> > >
> >
> > > > > > currently supported via the two level
> >
> > >
> >
> > > > > > committers of the new sink API: when received endOfInput(), the
> >
> > >
> >
> > > > Committer
> >
> > >
> >
> > > > > > wait for another checkpoint to
> >
> > >
> >
> > > > > > commits all the pending transactions and emit the list of files
> to
> >
> > > the
> >
> > >
> >
> > > > > > GlobalCommitter. The GlobalCommitter
> >
> > >
> >
> > > > > > would wait for another checkpoint to also write the metadata with
> >
> > 2pc
> >
> > >
> >
> > > > > > (Although sometimes 2pc is not needed
> >
> > >
> >
> > > > > > for writing metadata, it should be only an optimization and still
> >
> > >
> >
> > > > > requires
> >
> > >
> >
> > > > > > the Committer do commit before
> >
> > >
> >
> > > > > > notifying the global Committer. Also another note is
> >
> > GlobalCommitter
> >
> > > is
> >
> > >
> >
> > > > > > also added for some other cases
> >
> > >
> >
> > > > > > like some sinks want an commiter with dop = 1, like IceBergSink).
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > However, a more general issue to me is that currently we do not
> >
> > limit
> >
> > >
> >
> > > > > > users to not emit records in
> >
> > >
> >
> > > > > > notifyCheckpointComplete in the API level. The sink case could be
> >
> > >
> >
> > > > viewed
> >
> > >
> >
> > > > > > as a special case, but in addition
> >
> > >
> >
> > > > > > to this one, logically users could also implement their own cases
> >
> > > that
> >
> > >
> >
> > > > > > emits records in notifyCheckpointComplete.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > Best,
> >
> > >
> >
> > > > > > Yun
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > ------------------Original Mail ------------------
> >
> > >
> >
> > > > > > Sender:Arvid Heise
> >
> > >
> >
> > > > > > Send Date:Fri Feb 12 20:46:04 2021
> >
> > >
> >
> > > > > > Recipients:dev
> >
> > >
> >
> > > > > > CC:Yun Gao
> >
> > >
> >
> > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> >
> > >
> >
> > > > Finished
> >
> > >
> >
> > > > > > Hi Piotr,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > Thank you for raising your concern. Unfortunately, I do not have
> a
> >
> > >
> >
> > > > better
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > idea than doing closing of operators intermittently with
> >
> > checkpoints
> >
> > > (=
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > multiple last checkpoints).
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > However, two ideas on how to improve the overall user experience:
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > 1. If an operator is not relying on notifyCheckpointComplete, we
> >
> > can
> >
> > >
> >
> > > > > close
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > it faster (without waiting for a checkpoint). In general, I'd
> >
> > assume
> >
> > >
> >
> > > > that
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > almost all non-sinks behave that way.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > 2. We may increase the checkpointing frequency for the last
> >
> > >
> >
> > > > checkpoints.
> >
> > >
> >
> > > > > We
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > need to avoid overloading checkpoint storages and task managers,
> >
> > but
> >
> > > I
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > assume the more operators are closed, the lower the checkpointing
> >
> > >
> >
> > > > > interval
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > can be.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > For 1, I'd propose to add (name TBD):
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > default boolean StreamOperator#requiresFinalCheckpoint() {
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > return true;
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > }
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > This means all operators are conservatively (=slowly) closed. For
> >
> > > most
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > operators, we can then define their behavior by overriding in
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > AbstractUdfStreamOperator
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > @Override
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > return userFunction instanceof CheckpointListener;
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > }
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > This idea can be further refined in also adding
> >
> > > requiresFinalCheckpoint
> >
> > >
> >
> > > > > to
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > CheckpointListener to exclude all operators with UDFs that
> >
> > implement
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > CheckpointListener but do not need it for 2pc.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > @Override
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > return userFunction instanceof CheckpointListener &&
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > ((CheckpointListener) userFunction).requiresFinalCheckpoint();
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > }
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > That approach would also work for statebackends/snapshot
> strategies
> >
> > >
> >
> > > > that
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > require some 2pc.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > If we can contain it to the @PublicEvolving StreamOperator, it
> >
> > would
> >
> > > be
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > better of course.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > Best,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > Arvid
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > wrote:
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > Hey,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > I would like to raise a concern about implementation of the
> final
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > checkpoints taking into account operators/functions that are
> >
> > >
> >
> > > > > implementing
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > two phase commit (2pc) protocol for exactly-once processing
> with
> >
> > > some
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > external state (kept outside of the Flink). Primarily
> >
> > exactly-once
> >
> > >
> >
> > > > > sinks.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > First of all, as I understand it, this is not planned in the
> >
> > first
> >
> > >
> >
> > > > > > version
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > of this FLIP. I'm fine with that, however I would strongly
> >
> > > emphasize
> >
> > >
> >
> > > > > this
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > in every place we will be mentioning FLIP-147 efforts. This is
> >
> > >
> >
> > > > because
> >
> > >
> >
> > > > > > me,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > as a user, upon hearing "Flink supports checkpointing with
> >
> > bounded
> >
> > >
> >
> > > > > > inputs"
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > I would expect 2pc to work properly and to commit the external
> >
> > side
> >
> > >
> >
> > > > > > effects
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > upon finishing. As it is now, I (as a user) would be surprised
> >
> > > with a
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > silent data loss (of not committed trailing data). This is
> just a
> >
> > >
> >
> > > > > remark,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > that we need to attach this warning to every blog
> >
> > >
> >
> > > > > post/documentation/user
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > mailing list response related to "Support Checkpoints After
> Tasks
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > Finished". Also I would suggest to prioritize the follow up of
> >
> > >
> >
> > > > > supporting
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 2pc.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > Secondly, I think we are missing how difficult and problematic
> >
> > will
> >
> > >
> >
> > > > be
> >
> > >
> >
> > > > > > 2pc
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > support with the final checkpoint.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > For starters, keep in mind that currently 2pc can be
> implemented
> >
> > by
> >
> > >
> >
> > > > > users
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > using both `@Public` APIs as functions and `@PublicEvolving`
> >
> > >
> >
> > > > operators
> >
> > >
> >
> > > > > in
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > any place in the job graph. It's not limited to only the sinks.
> >
> > For
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > example users could easily implement the `AsynFunction` (for
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > `AsyncWaitOperator`) that is using 2pc based on the
> >
> > >
> >
> > > > > `CheckpointListener`
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > interface. I'm not saying it's common, probably just a tiny
> >
> > > minority
> >
> > >
> >
> > > > of
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > users are doing that (if any at all), but nevertheless that's
> >
> > >
> >
> > > > possible
> >
> > >
> >
> > > > > > and
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > currently (implicitly?) supported in Flink.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > Next complication is the support of bounded streams
> >
> > >
> >
> > > > (`BoundedOneInput`
> >
> > >
> >
> > > > > or
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > `BoundedMultiInput` interfaces) and the closing/shutdown
> >
> > procedure
> >
> > > of
> >
> > >
> >
> > > > > the
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > operators. Currently it works as follows:
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 1. `endOfInput` is called on the first operator in the chain
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 2. We quiesce the processing timers
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> for
> >
> > > the
> >
> > >
> >
> > > > > > first
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > operator, so no new timers will be triggered
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 3. We wait for the already fired timers to finish executing
> >
> > > (spinning
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > mailbox loop)
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 4. We are closing the first operator
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 5. We go to the next (second) operator in the chain and repeat
> >
> > the
> >
> > >
> >
> > > > > steps
> >
> > >
> >
> > > > > > 1.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > to 5.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > This is because operators can emit data after processing
> >
> > >
> >
> > > > `endOfInput`,
> >
> > >
> >
> > > > > > from
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > timers, async mailbox actions and inside the `close` method
> >
> > itself.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > Now the problem is to support the final checkpoint with 2pc, we
> >
> > > need
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > trigger `snapshotState` and `notifyCheckpointComplete` call at
> >
> > the
> >
> > >
> >
> > > > very
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > least only after `endOfInput` call on the operator. Probably
> the
> >
> > > best
> >
> > >
> >
> > > > > > place
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > would be in between steps 3. and 4. However that means, we
> would
> >
> > be
> >
> > >
> >
> > > > > > forced
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > to wait for steps 1. to 3. to finish, then wait for a next
> >
> > > checkpoint
> >
> > >
> >
> > > > > to
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > trigger AND complete, before finally closing the head operator,
> >
> > and
> >
> > >
> >
> > > > > only
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > then we can start closing the next operator in the chain:
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 1. `endOfInput` is called on the first operator in the chain
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 2. We quiesce the processing timers
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`)
> for
> >
> > > the
> >
> > >
> >
> > > > > > first
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > operator, so no new timers will be triggered
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 3. We wait for the already fired timers to finish executing
> >
> > > (spinning
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > mailbox loop)
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > *3b. We wait for one more checkpoint to trigger and for the
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > `notifyCheckpointComplete` RPC.*
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 4. We are closing the first operator
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 5. We go to the next (second) operator in the chain and repeat
> >
> > the
> >
> > >
> >
> > > > > steps
> >
> > >
> >
> > > > > > 1.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > to 5.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > That means, we can close one operator per successful
> checkpoint.
> >
> > To
> >
> > >
> >
> > > > > close
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > 10 operators, we would need 10 successful checkpoints.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > I was thinking about different approaches to this problem, and
> I
> >
> > >
> >
> > > > > couldn't
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > find any viable ones. All I could think of would break the
> >
> > current
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > `@Public` API and/or would be ugly/confusing for the users.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > For example a relatively simple solution, to introduce a
> >
> > `preClose`
> >
> > >
> >
> > > > or
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > `flush` method to the operators, with a contract that after
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > `flush`, operators would be forbidden from emitting more
> records,
> >
> > > so
> >
> > >
> >
> > > > > that
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > we can replace step 4. with this `flush` call, and then having
> a
> >
> > >
> >
> > > > single
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > checkpoint to finish 2pc for all of the operators inside the
> >
> > chain,
> >
> > >
> >
> > > > > > doesn't
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > work. Sheer fact of adding this `flush` method and changing the
> >
> > >
> >
> > > > > contract
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > would break the current API and Yun Gao has pointed out to me,
> >
> > that
> >
> > >
> >
> > > > we
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > either already support, or want to support operators that are
> >
> > >
> >
> > > > emitting
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > records from within the `notifyCheckpointComplete` call:
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > Yun Gao:
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > like with the new sink api there might be writer -> committer
> >
> > ->
> >
> > >
> >
> > > > > global
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > committer, the committer would need to wait for the last
> >
> > checkpoint
> >
> > >
> >
> > > > to
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > commit
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > the last piece of data, and after that it also need to emit
> the
> >
> > >
> >
> > > > list
> >
> > >
> >
> > > > > of
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > transactions get committed to global committer to do some
> >
> > >
> >
> > > > finalization
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > logic.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > So it wouldn't solve the problem (at least not fully).
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > I don't know if anyone has any better ideas how to solve this
> >
> > >
> >
> > > > problem?
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > Piotrek
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > pt., 15 sty 2021 o 14:57 Yun Gao
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > napisał(a):
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > Hi Aljoscha,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > I think so since we seems to do not have other divergence and
> >
> > new
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > objections now. I'll open the vote then. Very thanks!
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > Best,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > Yun
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > > ------------------------------------------------------------------
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > From:Aljoscha Krettek
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > Send Time:2021 Jan. 15 (Fri.) 21:24
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > To:dev
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After
> Tasks
> >
> > >
> >
> > > > > > Finished
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > Thanks for the summary! I think we can now move towards a
> >
> > [VOTE]
> >
> > >
> >
> > > > > > thread,
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > right?
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > On 2021/01/15 13:43, Yun Gao wrote:
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > >1) For the problem that the "new" root task coincidently
> >
> > > finished
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > >before getting triggered successfully, we have listed two
> >
> > > options
> >
> > >
> >
> > > > in
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > >the FLIP-147[1], for the first version, now we are not tend
> to
> >
> > > go
> >
> > >
> >
> > > > > with
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > >the first option that JM would re-compute and re-trigger new
> >
> > >
> >
> > > > sources
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > >when it realized some tasks are not triggered successfully.
> >
> > This
> >
> > >
> >
> > > > > > option
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > >would avoid the complexity of adding new PRC and duplicating
> >
> > > task
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > >states, and in average case it would not cause too much
> >
> > > overhead.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > You wrote "we are *not* tend to go with the first option",
> but
> >
> > I
> >
> > >
> >
> > > > > think
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > you meant wo write "we tend to *now* go with the first
> option",
> >
> > >
> >
> > > > > right?
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > That's also how it is in the FLIP, I just wanted to clarify
> for
> >
> > > the
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > > mailing list.
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > > >
> >
> > >
> >
> > > > >
> >
> > >
> >
> > > >
> >
> > >
> >
> > >
> >
> >
>

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

Posted by Kezhu Wang <ke...@gmail.com>.
Hi Yun,

The termination phase I drafted depends on a de-fact that
`notifyCheckpointComplete` is ignored currently after `close`. Thus, any
external materialization inside `close`(including `endInput`) or
`notifyCheckpointComplete` is either not guaranteed or not committed.

I tried to emphasize no-breaking changes and post-pone migration in later
releases for this termination phase. But now, I am kind of worry about
whether it will cause long-term maintenance hell.

Personally, I think allowing checkpoint after `close`(@Till proposed this
in FLINK-21133) could be the minimal change. But there are concerns:
* It will break some existing code possibly in a silent way.(@Piotr pointed
this already in FLINK-21133)
* I think the name `close` is kind of misleading. (@Piotr suggested
renaming this to `finish` to not break code silently in FLINK-21133)

> stop-with-savepoint --drain would wait for a specific savepoint

For stop-with-savepoint, the checkpoint barrier is already created or
received there.


Best,
Kezhu Wang

On February 25, 2021 at 21:02:30, Yun Gao (yungao.gy@aliyun.com) wrote:

Hi all,

Very thanks for the discussions!



A. Regarding how to avoid emitting records in notifyCheckpointComplete:

Currently the structure of a new sink is writer -> committer -> global
committer and the paralellism of global committer
must be one. By design it would be used in several cases:
1. writer -> committer: for normal sinks that write pending data in writer
and commit pending data in committer.
2. writer -> global committer: for sinks require the committer's
parallelism be 1, like iceberg sink which
relies on optimistic lock to commit data hopes to reduce the conflicts.
3. writer -> committer -> global committer: for sinks that also need to
write meta data (like _SUCCESS file
or add record in hive metastore) in global committer.

The third case would cause the cascade commit problem. To overcome this
problem, we would like to
1. Previously the global committer also support writing metadata with 2pc.
Now we disable this mode
and always rely on the property that writing metadata is repeatable. With
this limitation there should no
need of cascade commit, but the committer would still emit records in
notifyCheckpointComplete.
2. We further move global committer in the case 3 to the operator
coordinator. Then the committer would
not need to emit records, but communicates with its operator coordinator.

One core issue for using the OperatorCoordinator is how to keep the
communication between the operator
and the OperatorCoordinator exactly-once. Since the message is always from
the oeprator to the OperatorCoordinator
in this case, we would only need to bookkeep the message sent between the
OperatorCoordinator takes snapshot and
the Operator takes snapshot in the state of the Operator.

On how to achieve the change in detail we would still need some think, it
currently seems we would have to had
some modification to the current new sink api.



B. Regarding the stop-with-savepoint --drain

Very thanks @Piotr for the further explanation and now I realize I have
understand wrongly for the semantics of
stop-with-savepoint --drain. Now I think that the problem should be we
should also include the records produced in
`endOfInput()` and `close()` also in the last savepoint, am I correct? If
so, it seems we still have some undetermined options for
the lifecycle of the operator, like in Kezhu's proposal the close() happens
at last, but it seems close() might also emit records (
so now the operator are closed with op1's close() -> op2's endOfInput() ->
op2's close() -> op3's endOfinput -> ...) ?

And on the other side, as Kezhu has also proposed, perhapse we might have
the stop-with-savepoint --drain and normal exit in the same process,
but have slightly difference in that stop-with-savepoint --drain would wait
for a specific savepoint and in normal exit, the operator
might wait for arbitrary checkpoint. If we could achieve not emitting
records in notifyCheckpointComplete, stop-with-savepoint --drain could
be done with one savepoint, and for the normal exit, the operator would not
need to wait for other slow operators to exit.

Best,
Yun



------------------Original Mail ------------------
*Sender:*Kezhu Wang <ke...@gmail.com>
*Send Date:*Thu Feb 25 15:11:53 2021
*Recipients:*Flink Dev <de...@flink.apache.org>, Piotr Nowojski <
piotr.nowojski@gmail.com>
*CC:*Guowei Ma <gu...@gmail.com>, jingsonglee0@gmail.com <
jingsonglee0@gmail.com>
*Subject:*Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
Finished

> Hi all, thanks for driving this and especially Piotr for re-active this
>
> thread.
>
>
>
> First, for `notifyCheckpointComplete`, I have strong preference towards
>
> "shut down the dataflow
>
> pipeline with one checkpoint in total", so I tend to option dropping "send
>
> records" from
>
> `notifyCheckpointComplete` for next level committing in pipeline, if we
>
> ever support it. Without this,
>
> we are unable to stop a pipeline manually with all results materialized.
>
>
>
> Second, for shutdown unification of `stop-with-savepoint --drain` and
>
> FLIP-147, I draft following
>
> phase based on emerging proposals(mainly by Piotr and Till) in this thread
>
> and FLINK-21133.
>
>
>
> ```java
>
> // StreamTask termination phase
>
> finish()(Call StreamOperator.finish in chaining order)
>
> advanceToEndOfEventTime()(nop if for no-source inputs, this could also be
>
> done in input processor or finish ?)
>
> if (there-is-a-pending-terminate-savepoint) {
>
> triggerBarrierForDownStream();
>
> waitCheckpointComplete();
>
> } else if (require-two-phase-commit-in-shutdown) {
>
> waitFinalCheckpoint();
>
> waitCheckpointComplete();
>
> }
>
> // TODO: Ignore/Cancel all checkpoints from this point(or hit to
>
> coordinator in final checkpoint response ?).
>
> close();
>
> ```
>
>
>
> Branches in `if` could reside in different code paths, but the effect
>
> should be similar to above.
>
> The divergence reflects fact that "stop-with-savepoint --drain" is
>
> triggered already while we need
>
> to wait for final checkpoint in natural input exhausting.
>
>
>
> With carefully chosen default functions, we will not break existing
>
> interfaces.
>
> * `StreamOperator.finish`
>
> * `RichFunction.finish`
>
>
>
> For existing codes:
>
> 1. "stop-with-savepoint" with no migration, it should behaves as before.
>
> 2. "require-two-phase-commit-in-shutdown" evaluates to false and no
>
> migration, it is same as before.
>
> 3. "require-two-phase-commit-in-shutdown" evaluates to true and no
>
> migration, records in `close` will
>
> lose.
>
>
>
> For case#3:
>
> A. No external materialization in `StreamOperator.close`: Nothing to lose.
>
> B. External materialization only in `StreamOperator.close`: I don't think
>
> Flink ever claimed that there is
>
> any guarantee for this situation.
>
> C. External materialization in `notifyCheckpointComplete` from
>
> `StreamOperator.close`: But due to fact that
>
> `notifyCheckpointComplete` was ignored after operator
>
> closed(FLINK-16383), so there will be no external
>
> materialization to lose.
>
>
>
> Then, we could recommend users to migrate possible “end-of-stream-flushing”
>
> from “close" to “finish”.
>
>
>
> Best,
>
> Kezhu Wang
>
>
>
>
>
> On February 24, 2021 at 23:47:36, Piotr Nowojski (piotr.nowojski@gmail.com
> )
>
> wrote:
>
>
>
> Thanks for the reponses Guowei and Yun,
>
>
>
> Could you elaborate more/remind me, what does it mean to replace emitting
>
> results from the `notifyCheckpointComplete` with `OperatorCoordinator`
>
> approach?
>
>
>
> About the discussion in FLINK-21133 and how it relates to FLIP-147. You are
>
> right Yun gao, that in case of `stop-with-savepoint --drain` the whole job
>
> finishes, while in FLIP-147 some of the sources can work for arbitrarily
>
> long time after others have already finished. However from the runtime
>
> perspective, especially how to close a Task and it's operators, there is no
>
> difference between those two cases. In both cases, we need to end input,
>
> shut down processing time timers and commit external side effects (two
>
> phase commit support) before exiting the task. That's how the discussion
>
> about the behaviour of "stop-with-savepoint" was connected with FLIP-147.
>
>
>
> Currently on master, "stop-with-savepoint --drain" drains/flushes buffered
>
> records and deals correctly with timers, but all of that happens AFTER
>
> savepoint was completed. So any records flushed from the operators during
>
> endOfInput/close/shutting down processing timers are never committed to
>
> external systems. This is exactly the same problem as the "two phase
>
> commit" problem of FLIP-147, that should have the same solution and it
>
> should be solved at the same time. For example if we go with the *very
>
> undesirable* "one closed operator per one completed checkpoint/savepoint",
>
> in both cases CheckpointCoordinator, Scheduler and Task would need to keep
>
> the task alive and keep triggering checkpoints for that task, until all
>
> operators in the operator chain are closed (one closed operator per one
>
> completed checkpoint).
>
>
>
> Piotrek
>
>
>
>
>
>
>
> śr., 24 lut 2021 o 11:30 Yun Gao napisał(a):
>
>
>
> > Hi Till, Guowei,
>
> >
>
> > Very thanks for initiating the disucssion and the deep thoughts!
>
> >
>
> > For the notifyCheckpointComplete, I also agree we could try to avoid
>
> > emitting new records in notifyCheckpointComplete via using
>
> > OperatorCoordinator
>
> > for new sink API. Besides, the hive sink might also need some
>
> modification
>
> > for it also emits records in notifyCheckpointComplete.
>
> >
>
> > For unifying the process of stopping with savepoint and finished due to
>
> > all records
>
> > are processed, I also agree with that unifying would always be better if
>
> > we could achieve,
>
> > but I'm still not fully catch up with the implementation: Based on the
>
> > discussion in FLINK-21133,
>
> > my understanding is that for stopping with savepoint, now we want to
>
> first
>
> > stop the source, then we
>
> > trigger a savepoint, and after the source received
>
> > notifyCheckpointComplete, the source would
>
> > start emitting EndOfPartitionEvent to finish the job, am I correct ?
>
> >
>
> > For normal finish, a difference to me might be if we have multiple
>
> > sources, we could not guarantee
>
> > when the sources are to finish. We might have one source run one 1 minute
>
> > and another one run for
>
> > 1 hour. To unify with the process with stop with savepoint, we might need
>
> > to hold the fast source until
>
> > all the sources are finished? An coordinator would be introduced to count
>
> > the number of sources
>
> > runing and trigger the final savepoint / checkpoint. For the extreme
>
> > cases, if we have both bounded and
>
> > unbounded sources, we might only count how much bounded source are
>
> > remaining ? And if all the bounded
>
> > sources are finished we would trigger the special checkpoint. After all
>
> > the bounded part of the graph are
>
> > finished, the the remaining part could still do checkpoint and commit
>
> data
>
> > with FLIP-147.
>
> >
>
> > Best,
>
> > Yun
>
> >
>
> >
>
> >
>
> >
>
> > ------------------Original Mail ------------------
>
> > Sender:Guowei Ma
>
> > Send Date:Wed Feb 24 17:35:36 2021
>
> > Recipients:dev
>
> > CC:Arvid Heise
>
> > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
> > Hi, Till
>
> >
>
> > Thank you very much for your careful consideration
>
> >
>
> >
>
> >
>
> > *1. Emit records in `NotifyCheckpointComplete`.*
>
> >
>
> > Sorry for making you misunderstanding because of my expression. I just
>
> >
>
> > want to say the current interface does not prevent users from doing it.
>
> >
>
> > From the perspective of the new sink api, we might not depend on emitting
>
> >
>
> > records in `NotifyCheckpointComplete`, like using `OperatorCoordinator`
>
> >
>
> > instead.
>
> >
>
> >
>
> >
>
> >
>
> >
>
> > *2. What does the FLIP-147 guarantee?*I think initially this FLIP want to
>
> >
>
> > achieve two targets:
>
> >
>
> > 1. Tasks/Operators exit correctly (as you mentioned the lifecycle of a
>
> >
>
> > Task/StreamTask/StreamOperator.).
>
> >
>
> > 2. Continue to trigger checkpoint after some tasks for mixed jobs.
>
> >
>
> >
>
> >
>
> > I think the first thing is related to the discussion in FLINK-21133. If I
>
> >
>
> > understand correctly, in addition to supporting the tasks / operators to
>
> >
>
> > exit correctly, now we also want to unify the process of the tasks and
>
> >
>
> > operators for savepoint / finish.
>
> >
>
> > I think the second thing is orthogonal to the FLINK-21133 because there
>
> are
>
> >
>
> > topologies that have both the bounded and unbounded input.
>
> >
>
> >
>
> >
>
> > *3. How to unify the operator exit process of FLIP-147 with
>
> >
>
> > stop-with-savepoint?*
>
> >
>
> > I am not very sure about how to do it yet. But if I understand the
>
> >
>
> > discussion in the jira correctly it needs to introduce some logic into
>
> >
>
> > `CheckpointCoordinator`, which responses for triggering “the unified
>
> >
>
> > operator exit process”. Am I correct?
>
> >
>
> >
>
> >
>
> > Best,
>
> >
>
> > Guowei
>
> >
>
> >
>
> >
>
> >
>
> >
>
> > On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann wrote:
>
> >
>
> >
>
> >
>
> > > Thanks for the explanation Yun and Guowei. I have to admit that I do
>
> not
>
> >
>
> > > fully understand why this is strictly required but I think that we are
>
> >
>
> > > touching two very important aspects which might have far fetching
>
> >
>
> > > consequences for how Flink works:
>
> >
>
> > >
>
> >
>
> > > 1) Do we want to allow that multiple checkpoints are required to
>
> >
>
> > > materialize results?
>
> >
>
> > > 2) Do we want to allow to emit records in notifyCheckpointComplete?
>
> >
>
> > >
>
> >
>
> > > For 1) I am not sure whether this has been discussed within the
>
> community
>
> >
>
> > > sufficiently. Requiring multiple checkpoints to materialize a result
>
> >
>
> > > because of multi level committers has the consequence that we increase
>
> > the
>
> >
>
> > > latency from checkpoint interval to #levels * checkpoint interval.
>
> >
>
> > > Moreover, having to drain the pipeline in multiple steps, would break
>
> the
>
> >
>
> > > stop-with-savepoint --drain because which savepoint do you report to
>
> the
>
> >
>
> > > user?
>
> >
>
> > >
>
> >
>
> > > For 2) allowing to send records after the final
>
> notifyCheckpointComplete
>
> >
>
> > > will effectively mean that we need to shut down a topology in multiple
>
> >
>
> > > steps (in the worst case one operator per checkpoint). This would be a
>
> >
>
> > > strong argument for not allowing this to me. The fact that users can
>
> send
>
> >
>
> > > records after the notifyCheckpointComplete is more by accident than by
>
> >
>
> > > design. I think we should make this a very deliberate decision and in
>
> > doubt
>
> >
>
> > > I would be in favour of a more restrictive model unless there is a very
>
> >
>
> > > good reason why this should be supported.
>
> >
>
> > >
>
> >
>
> > > Taking also the discussion in FLINK-21133 [1] into account, it seems to
>
> > me
>
> >
>
> > > that we haven't really understood what kind of guarantees we want to
>
> give
>
> >
>
> > > to our users and how the final checkpoint should exactly work. I
>
> > understand
>
> >
>
> > > that this is not included in the first scope of FLIP-147 but I think
>
> this
>
> >
>
> > > is so important that we should figure this out asap. Also because the
>
> > exact
>
> >
>
> > > shut down behaviour will have to be aligned with the lifecycle of a
>
> >
>
> > > Task/StreamTask/StreamOperator. And last but not least because other
>
> >
>
> > > features such as the new sink API start building upon a shut down model
>
> >
>
> > > which has not been fully understood/agreed upon.
>
> >
>
> > >
>
> >
>
> > > [1] https://issues.apache.org/jira/browse/FLINK-21133
>
> >
>
> > >
>
> >
>
> > > Cheers,
>
> >
>
> > > Till
>
> >
>
> > >
>
> >
>
> > > On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma wrote:
>
> >
>
> > >
>
> >
>
> > > > Thanks Yun for the detailed explanation.
>
> >
>
> > > > A simple supplementary explanation about the sink case: Maybe we
>
> could
>
> >
>
> > > use
>
> >
>
> > > > `OperatorCoordinator` to avoid sending the element to the downstream
>
> >
>
> > > > operator.
>
> >
>
> > > > But I agree we could not limit the users not to emit records in the
>
> >
>
> > > > `notiyCheckpointComplete`.
>
> >
>
> > > >
>
> >
>
> > > > Best,
>
> >
>
> > > > Guowei
>
> >
>
> > > >
>
> >
>
> > > >
>
> >
>
> > > > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao
>
> >
>
> > > > wrote:
>
> >
>
> > > >
>
> >
>
> > > > > Hi all,
>
> >
>
> > > > >
>
> >
>
> > > > > I'd like to first detail the issue with emitting records in
>
> >
>
> > > > > notifyCheckpointComplete for context. For specific usage,
>
> >
>
> > > > > an example would be for sink, it might want to write some metadata
>
> >
>
> > > after
>
> >
>
> > > > > all the transactions are committed
>
> >
>
> > > > > (like write a marker file _SUCCESS to the output directory). This
>
> > case
>
> >
>
> > > is
>
> >
>
> > > > > currently supported via the two level
>
> >
>
> > > > > committers of the new sink API: when received endOfInput(), the
>
> >
>
> > > Committer
>
> >
>
> > > > > wait for another checkpoint to
>
> >
>
> > > > > commits all the pending transactions and emit the list of files to
>
> > the
>
> >
>
> > > > > GlobalCommitter. The GlobalCommitter
>
> >
>
> > > > > would wait for another checkpoint to also write the metadata with
>
> 2pc
>
> >
>
> > > > > (Although sometimes 2pc is not needed
>
> >
>
> > > > > for writing metadata, it should be only an optimization and still
>
> >
>
> > > > requires
>
> >
>
> > > > > the Committer do commit before
>
> >
>
> > > > > notifying the global Committer. Also another note is
>
> GlobalCommitter
>
> > is
>
> >
>
> > > > > also added for some other cases
>
> >
>
> > > > > like some sinks want an commiter with dop = 1, like IceBergSink).
>
> >
>
> > > > >
>
> >
>
> > > > > However, a more general issue to me is that currently we do not
>
> limit
>
> >
>
> > > > > users to not emit records in
>
> >
>
> > > > > notifyCheckpointComplete in the API level. The sink case could be
>
> >
>
> > > viewed
>
> >
>
> > > > > as a special case, but in addition
>
> >
>
> > > > > to this one, logically users could also implement their own cases
>
> > that
>
> >
>
> > > > > emits records in notifyCheckpointComplete.
>
> >
>
> > > > >
>
> >
>
> > > > > Best,
>
> >
>
> > > > > Yun
>
> >
>
> > > > >
>
> >
>
> > > > > ------------------Original Mail ------------------
>
> >
>
> > > > > Sender:Arvid Heise
>
> >
>
> > > > > Send Date:Fri Feb 12 20:46:04 2021
>
> >
>
> > > > > Recipients:dev
>
> >
>
> > > > > CC:Yun Gao
>
> >
>
> > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
>
> >
>
> > > Finished
>
> >
>
> > > > > Hi Piotr,
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > Thank you for raising your concern. Unfortunately, I do not have a
>
> >
>
> > > better
>
> >
>
> > > > >
>
> >
>
> > > > > idea than doing closing of operators intermittently with
>
> checkpoints
>
> > (=
>
> >
>
> > > > >
>
> >
>
> > > > > multiple last checkpoints).
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > However, two ideas on how to improve the overall user experience:
>
> >
>
> > > > >
>
> >
>
> > > > > 1. If an operator is not relying on notifyCheckpointComplete, we
>
> can
>
> >
>
> > > > close
>
> >
>
> > > > >
>
> >
>
> > > > > it faster (without waiting for a checkpoint). In general, I'd
>
> assume
>
> >
>
> > > that
>
> >
>
> > > > >
>
> >
>
> > > > > almost all non-sinks behave that way.
>
> >
>
> > > > >
>
> >
>
> > > > > 2. We may increase the checkpointing frequency for the last
>
> >
>
> > > checkpoints.
>
> >
>
> > > > We
>
> >
>
> > > > >
>
> >
>
> > > > > need to avoid overloading checkpoint storages and task managers,
>
> but
>
> > I
>
> >
>
> > > > >
>
> >
>
> > > > > assume the more operators are closed, the lower the checkpointing
>
> >
>
> > > > interval
>
> >
>
> > > > >
>
> >
>
> > > > > can be.
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > For 1, I'd propose to add (name TBD):
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > default boolean StreamOperator#requiresFinalCheckpoint() {
>
> >
>
> > > > >
>
> >
>
> > > > > return true;
>
> >
>
> > > > >
>
> >
>
> > > > > }
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > This means all operators are conservatively (=slowly) closed. For
>
> > most
>
> >
>
> > > > >
>
> >
>
> > > > > operators, we can then define their behavior by overriding in
>
> >
>
> > > > >
>
> >
>
> > > > > AbstractUdfStreamOperator
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > @Override
>
> >
>
> > > > >
>
> >
>
> > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
>
> >
>
> > > > >
>
> >
>
> > > > > return userFunction instanceof CheckpointListener;
>
> >
>
> > > > >
>
> >
>
> > > > > }
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > This idea can be further refined in also adding
>
> > requiresFinalCheckpoint
>
> >
>
> > > > to
>
> >
>
> > > > >
>
> >
>
> > > > > CheckpointListener to exclude all operators with UDFs that
>
> implement
>
> >
>
> > > > >
>
> >
>
> > > > > CheckpointListener but do not need it for 2pc.
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > @Override
>
> >
>
> > > > >
>
> >
>
> > > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
>
> >
>
> > > > >
>
> >
>
> > > > > return userFunction instanceof CheckpointListener &&
>
> >
>
> > > > >
>
> >
>
> > > > > ((CheckpointListener) userFunction).requiresFinalCheckpoint();
>
> >
>
> > > > >
>
> >
>
> > > > > }
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > That approach would also work for statebackends/snapshot strategies
>
> >
>
> > > that
>
> >
>
> > > > >
>
> >
>
> > > > > require some 2pc.
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > If we can contain it to the @PublicEvolving StreamOperator, it
>
> would
>
> > be
>
> >
>
> > > > >
>
> >
>
> > > > > better of course.
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > Best,
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > Arvid
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
>
> >
>
> > > > >
>
> >
>
> > > > > wrote:
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > Hey,
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > I would like to raise a concern about implementation of the final
>
> >
>
> > > > >
>
> >
>
> > > > > > checkpoints taking into account operators/functions that are
>
> >
>
> > > > implementing
>
> >
>
> > > > >
>
> >
>
> > > > > > two phase commit (2pc) protocol for exactly-once processing with
>
> > some
>
> >
>
> > > > >
>
> >
>
> > > > > > external state (kept outside of the Flink). Primarily
>
> exactly-once
>
> >
>
> > > > sinks.
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > First of all, as I understand it, this is not planned in the
>
> first
>
> >
>
> > > > > version
>
> >
>
> > > > >
>
> >
>
> > > > > > of this FLIP. I'm fine with that, however I would strongly
>
> > emphasize
>
> >
>
> > > > this
>
> >
>
> > > > >
>
> >
>
> > > > > > in every place we will be mentioning FLIP-147 efforts. This is
>
> >
>
> > > because
>
> >
>
> > > > > me,
>
> >
>
> > > > >
>
> >
>
> > > > > > as a user, upon hearing "Flink supports checkpointing with
>
> bounded
>
> >
>
> > > > > inputs"
>
> >
>
> > > > >
>
> >
>
> > > > > > I would expect 2pc to work properly and to commit the external
>
> side
>
> >
>
> > > > > effects
>
> >
>
> > > > >
>
> >
>
> > > > > > upon finishing. As it is now, I (as a user) would be surprised
>
> > with a
>
> >
>
> > > > >
>
> >
>
> > > > > > silent data loss (of not committed trailing data). This is just a
>
> >
>
> > > > remark,
>
> >
>
> > > > >
>
> >
>
> > > > > > that we need to attach this warning to every blog
>
> >
>
> > > > post/documentation/user
>
> >
>
> > > > >
>
> >
>
> > > > > > mailing list response related to "Support Checkpoints After Tasks
>
> >
>
> > > > >
>
> >
>
> > > > > > Finished". Also I would suggest to prioritize the follow up of
>
> >
>
> > > > supporting
>
> >
>
> > > > >
>
> >
>
> > > > > > 2pc.
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > Secondly, I think we are missing how difficult and problematic
>
> will
>
> >
>
> > > be
>
> >
>
> > > > > 2pc
>
> >
>
> > > > >
>
> >
>
> > > > > > support with the final checkpoint.
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > For starters, keep in mind that currently 2pc can be implemented
>
> by
>
> >
>
> > > > users
>
> >
>
> > > > >
>
> >
>
> > > > > > using both `@Public` APIs as functions and `@PublicEvolving`
>
> >
>
> > > operators
>
> >
>
> > > > in
>
> >
>
> > > > >
>
> >
>
> > > > > > any place in the job graph. It's not limited to only the sinks.
>
> For
>
> >
>
> > > > >
>
> >
>
> > > > > > example users could easily implement the `AsynFunction` (for
>
> >
>
> > > > >
>
> >
>
> > > > > > `AsyncWaitOperator`) that is using 2pc based on the
>
> >
>
> > > > `CheckpointListener`
>
> >
>
> > > > >
>
> >
>
> > > > > > interface. I'm not saying it's common, probably just a tiny
>
> > minority
>
> >
>
> > > of
>
> >
>
> > > > >
>
> >
>
> > > > > > users are doing that (if any at all), but nevertheless that's
>
> >
>
> > > possible
>
> >
>
> > > > > and
>
> >
>
> > > > >
>
> >
>
> > > > > > currently (implicitly?) supported in Flink.
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > Next complication is the support of bounded streams
>
> >
>
> > > (`BoundedOneInput`
>
> >
>
> > > > or
>
> >
>
> > > > >
>
> >
>
> > > > > > `BoundedMultiInput` interfaces) and the closing/shutdown
>
> procedure
>
> > of
>
> >
>
> > > > the
>
> >
>
> > > > >
>
> >
>
> > > > > > operators. Currently it works as follows:
>
> >
>
> > > > >
>
> >
>
> > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
>
> >
>
> > > > >
>
> >
>
> > > > > > 1. `endOfInput` is called on the first operator in the chain
>
> >
>
> > > > >
>
> >
>
> > > > > > 2. We quiesce the processing timers
>
> >
>
> > > > >
>
> >
>
> > > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for
>
> > the
>
> >
>
> > > > > first
>
> >
>
> > > > >
>
> >
>
> > > > > > operator, so no new timers will be triggered
>
> >
>
> > > > >
>
> >
>
> > > > > > 3. We wait for the already fired timers to finish executing
>
> > (spinning
>
> >
>
> > > > >
>
> >
>
> > > > > > mailbox loop)
>
> >
>
> > > > >
>
> >
>
> > > > > > 4. We are closing the first operator
>
> >
>
> > > > >
>
> >
>
> > > > > > 5. We go to the next (second) operator in the chain and repeat
>
> the
>
> >
>
> > > > steps
>
> >
>
> > > > > 1.
>
> >
>
> > > > >
>
> >
>
> > > > > > to 5.
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > This is because operators can emit data after processing
>
> >
>
> > > `endOfInput`,
>
> >
>
> > > > > from
>
> >
>
> > > > >
>
> >
>
> > > > > > timers, async mailbox actions and inside the `close` method
>
> itself.
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > Now the problem is to support the final checkpoint with 2pc, we
>
> > need
>
> >
>
> > > > >
>
> >
>
> > > > > > trigger `snapshotState` and `notifyCheckpointComplete` call at
>
> the
>
> >
>
> > > very
>
> >
>
> > > > >
>
> >
>
> > > > > > least only after `endOfInput` call on the operator. Probably the
>
> > best
>
> >
>
> > > > > place
>
> >
>
> > > > >
>
> >
>
> > > > > > would be in between steps 3. and 4. However that means, we would
>
> be
>
> >
>
> > > > > forced
>
> >
>
> > > > >
>
> >
>
> > > > > > to wait for steps 1. to 3. to finish, then wait for a next
>
> > checkpoint
>
> >
>
> > > > to
>
> >
>
> > > > >
>
> >
>
> > > > > > trigger AND complete, before finally closing the head operator,
>
> and
>
> >
>
> > > > only
>
> >
>
> > > > >
>
> >
>
> > > > > > then we can start closing the next operator in the chain:
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
>
> >
>
> > > > >
>
> >
>
> > > > > > 1. `endOfInput` is called on the first operator in the chain
>
> >
>
> > > > >
>
> >
>
> > > > > > 2. We quiesce the processing timers
>
> >
>
> > > > >
>
> >
>
> > > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for
>
> > the
>
> >
>
> > > > > first
>
> >
>
> > > > >
>
> >
>
> > > > > > operator, so no new timers will be triggered
>
> >
>
> > > > >
>
> >
>
> > > > > > 3. We wait for the already fired timers to finish executing
>
> > (spinning
>
> >
>
> > > > >
>
> >
>
> > > > > > mailbox loop)
>
> >
>
> > > > >
>
> >
>
> > > > > > *3b. We wait for one more checkpoint to trigger and for the
>
> >
>
> > > > >
>
> >
>
> > > > > > `notifyCheckpointComplete` RPC.*
>
> >
>
> > > > >
>
> >
>
> > > > > > 4. We are closing the first operator
>
> >
>
> > > > >
>
> >
>
> > > > > > 5. We go to the next (second) operator in the chain and repeat
>
> the
>
> >
>
> > > > steps
>
> >
>
> > > > > 1.
>
> >
>
> > > > >
>
> >
>
> > > > > > to 5.
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > That means, we can close one operator per successful checkpoint.
>
> To
>
> >
>
> > > > close
>
> >
>
> > > > >
>
> >
>
> > > > > > 10 operators, we would need 10 successful checkpoints.
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > I was thinking about different approaches to this problem, and I
>
> >
>
> > > > couldn't
>
> >
>
> > > > >
>
> >
>
> > > > > > find any viable ones. All I could think of would break the
>
> current
>
> >
>
> > > > >
>
> >
>
> > > > > > `@Public` API and/or would be ugly/confusing for the users.
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > For example a relatively simple solution, to introduce a
>
> `preClose`
>
> >
>
> > > or
>
> >
>
> > > > >
>
> >
>
> > > > > > `flush` method to the operators, with a contract that after
>
> >
>
> > > > >
>
> >
>
> > > > > > `flush`, operators would be forbidden from emitting more records,
>
> > so
>
> >
>
> > > > that
>
> >
>
> > > > >
>
> >
>
> > > > > > we can replace step 4. with this `flush` call, and then having a
>
> >
>
> > > single
>
> >
>
> > > > >
>
> >
>
> > > > > > checkpoint to finish 2pc for all of the operators inside the
>
> chain,
>
> >
>
> > > > > doesn't
>
> >
>
> > > > >
>
> >
>
> > > > > > work. Sheer fact of adding this `flush` method and changing the
>
> >
>
> > > > contract
>
> >
>
> > > > >
>
> >
>
> > > > > > would break the current API and Yun Gao has pointed out to me,
>
> that
>
> >
>
> > > we
>
> >
>
> > > > >
>
> >
>
> > > > > > either already support, or want to support operators that are
>
> >
>
> > > emitting
>
> >
>
> > > > >
>
> >
>
> > > > > > records from within the `notifyCheckpointComplete` call:
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > > Yun Gao:
>
> >
>
> > > > >
>
> >
>
> > > > > > > like with the new sink api there might be writer -> committer
>
> ->
>
> >
>
> > > > global
>
> >
>
> > > > >
>
> >
>
> > > > > > committer, the committer would need to wait for the last
>
> checkpoint
>
> >
>
> > > to
>
> >
>
> > > > >
>
> >
>
> > > > > > commit
>
> >
>
> > > > >
>
> >
>
> > > > > > > the last piece of data, and after that it also need to emit the
>
> >
>
> > > list
>
> >
>
> > > > of
>
> >
>
> > > > >
>
> >
>
> > > > > > transactions get committed to global committer to do some
>
> >
>
> > > finalization
>
> >
>
> > > > >
>
> >
>
> > > > > > logic.
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > So it wouldn't solve the problem (at least not fully).
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > I don't know if anyone has any better ideas how to solve this
>
> >
>
> > > problem?
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > Piotrek
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > pt., 15 sty 2021 o 14:57 Yun Gao
>
> >
>
> > > > >
>
> >
>
> > > > > > napisał(a):
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > > Hi Aljoscha,
>
> >
>
> > > > >
>
> >
>
> > > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > > I think so since we seems to do not have other divergence and
>
> new
>
> >
>
> > > > >
>
> >
>
> > > > > > > objections now. I'll open the vote then. Very thanks!
>
> >
>
> > > > >
>
> >
>
> > > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > > Best,
>
> >
>
> > > > >
>
> >
>
> > > > > > > Yun
>
> >
>
> > > > >
>
> >
>
> > > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > >
>
> > ------------------------------------------------------------------
>
> >
>
> > > > >
>
> >
>
> > > > > > > From:Aljoscha Krettek
>
> >
>
> > > > >
>
> >
>
> > > > > > > Send Time:2021 Jan. 15 (Fri.) 21:24
>
> >
>
> > > > >
>
> >
>
> > > > > > > To:dev
>
> >
>
> > > > >
>
> >
>
> > > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
>
> >
>
> > > > > Finished
>
> >
>
> > > > >
>
> >
>
> > > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > > Thanks for the summary! I think we can now move towards a
>
> [VOTE]
>
> >
>
> > > > > thread,
>
> >
>
> > > > >
>
> >
>
> > > > > > > right?
>
> >
>
> > > > >
>
> >
>
> > > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > > On 2021/01/15 13:43, Yun Gao wrote:
>
> >
>
> > > > >
>
> >
>
> > > > > > > >1) For the problem that the "new" root task coincidently
>
> > finished
>
> >
>
> > > > >
>
> >
>
> > > > > > > >before getting triggered successfully, we have listed two
>
> > options
>
> >
>
> > > in
>
> >
>
> > > > >
>
> >
>
> > > > > > > >the FLIP-147[1], for the first version, now we are not tend to
>
> > go
>
> >
>
> > > > with
>
> >
>
> > > > >
>
> >
>
> > > > > > > >the first option that JM would re-compute and re-trigger new
>
> >
>
> > > sources
>
> >
>
> > > > >
>
> >
>
> > > > > > > >when it realized some tasks are not triggered successfully.
>
> This
>
> >
>
> > > > > option
>
> >
>
> > > > >
>
> >
>
> > > > > > > >would avoid the complexity of adding new PRC and duplicating
>
> > task
>
> >
>
> > > > >
>
> >
>
> > > > > > > >states, and in average case it would not cause too much
>
> > overhead.
>
> >
>
> > > > >
>
> >
>
> > > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > > You wrote "we are *not* tend to go with the first option", but
>
> I
>
> >
>
> > > > think
>
> >
>
> > > > >
>
> >
>
> > > > > > > you meant wo write "we tend to *now* go with the first option",
>
> >
>
> > > > right?
>
> >
>
> > > > >
>
> >
>
> > > > > > > That's also how it is in the FLIP, I just wanted to clarify for
>
> > the
>
> >
>
> > > > >
>
> >
>
> > > > > > > mailing list.
>
> >
>
> > > > >
>
> >
>
> > > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > > >
>
> >
>
> > > > >
>
> >
>
> > > > >
>
> >
>
> > > >
>
> >
>
> > >
>
> >
>
> >
>
>

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi all,

Very thanks for the discussions!



A. Regarding how to avoid emitting records in notifyCheckpointComplete:

Currently the structure of a new sink is writer -> committer -> global committer and the paralellism of global committer 
must be one. By design it would be used in several cases:
1. writer -> committer: for normal sinks that write pending data in writer and commit pending data in committer.
2. writer -> global committer: for sinks require the committer's parallelism be 1, like iceberg sink which
relies on optimistic lock to commit data hopes to reduce the conflicts.
3. writer -> committer -> global committer: for sinks that also need to write meta data (like _SUCCESS file
or add record in hive metastore) in global committer.

The third case would cause the cascade commit problem. To overcome this problem, we would like to
1. Previously the global committer also support writing metadata with 2pc. Now we disable this mode
and always rely on the property that writing metadata is repeatable. With this limitation there should no 
need of cascade commit, but the committer would still emit records in notifyCheckpointComplete.
2. We further move global committer in the case 3 to the operator coordinator. Then the committer would
not need to emit records, but communicates with its operator coordinator.

One core issue for using the OperatorCoordinator is how to keep the communication between the operator 
and the OperatorCoordinator exactly-once. Since the message is always from the oeprator to the OperatorCoordinator
in this case, we would only need to bookkeep the message sent between the OperatorCoordinator takes snapshot and
the Operator takes snapshot in the state of the Operator. 

On how to achieve the change in detail we would still need some think, it currently seems we would have to had
some modification to the current new sink api.



B. Regarding the stop-with-savepoint --drain

Very thanks @Piotr for the further explanation and now I realize I have understand wrongly for the semantics of 
stop-with-savepoint --drain. Now I think that the problem should be we should also include the records produced in
`endOfInput()` and `close()` also in the last savepoint, am I correct? If so, it seems we still have some undetermined options for
the lifecycle of the operator, like in Kezhu's proposal the close() happens at last, but it seems close() might also emit records (
so now the operator are closed with op1's close() -> op2's endOfInput() -> op2's close() -> op3's endOfinput -> ...) ?

And on the other side, as Kezhu has also proposed, perhapse we might have the stop-with-savepoint --drain and normal exit in the same process, 
but have slightly difference in that stop-with-savepoint --drain would wait for a specific savepoint and in normal exit, the operator
might wait for arbitrary checkpoint. If we could achieve not emitting records in notifyCheckpointComplete, stop-with-savepoint --drain could
be done with one savepoint, and for the normal exit, the operator would not need to wait for other slow operators to exit. 

Best,
Yun




 ------------------Original Mail ------------------
Sender:Kezhu Wang <ke...@gmail.com>
Send Date:Thu Feb 25 15:11:53 2021
Recipients:Flink Dev <de...@flink.apache.org>, Piotr Nowojski <pi...@gmail.com>
CC:Guowei Ma <gu...@gmail.com>, jingsonglee0@gmail.com <ji...@gmail.com>
Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
Hi all, thanks for driving this and especially Piotr for re-active this

thread.



First, for `notifyCheckpointComplete`, I have strong preference towards

"shut down the dataflow

pipeline with one checkpoint in total", so I tend to option dropping "send

records" from

`notifyCheckpointComplete` for next level committing in pipeline, if we

ever support it. Without this,

we are unable to stop a pipeline manually with all results materialized.



Second, for shutdown unification of `stop-with-savepoint --drain` and

FLIP-147, I draft following

phase based on emerging proposals(mainly by Piotr and Till) in this thread

and FLINK-21133.



```java

// StreamTask termination phase

finish()(Call StreamOperator.finish in chaining order)

advanceToEndOfEventTime()(nop if for no-source inputs, this could also be

done in input processor or finish ?)

if (there-is-a-pending-terminate-savepoint) {

 triggerBarrierForDownStream();

 waitCheckpointComplete();

} else if (require-two-phase-commit-in-shutdown) {

 waitFinalCheckpoint();

 waitCheckpointComplete();

}

// TODO: Ignore/Cancel all checkpoints from this point(or hit to

coordinator in final checkpoint response ?).

close();

```



Branches in `if` could reside in different code paths, but the effect

should be similar to above.

The divergence reflects fact that "stop-with-savepoint --drain" is

triggered already while we need

to wait for final checkpoint in natural input exhausting.



With carefully chosen default functions, we will not break existing

interfaces.

* `StreamOperator.finish`

* `RichFunction.finish`



For existing codes:

1. "stop-with-savepoint" with no migration, it should behaves as before.

2. "require-two-phase-commit-in-shutdown" evaluates to false and no

migration, it is same as before.

3. "require-two-phase-commit-in-shutdown" evaluates to true and no

migration, records in `close` will

 lose.



For case#3:

A. No external materialization in `StreamOperator.close`: Nothing to lose.

B. External materialization only in `StreamOperator.close`: I don't think

Flink ever claimed that there is

 any guarantee for this situation.

C. External materialization in `notifyCheckpointComplete` from

`StreamOperator.close`: But due to fact that

 `notifyCheckpointComplete` was ignored after operator

closed(FLINK-16383), so there will be no external

 materialization to lose.



Then, we could recommend users to migrate possible “end-of-stream-flushing”

from “close" to “finish”.



Best,

Kezhu Wang





On February 24, 2021 at 23:47:36, Piotr Nowojski (piotr.nowojski@gmail.com)

wrote:



Thanks for the reponses Guowei and Yun,



Could you elaborate more/remind me, what does it mean to replace emitting

results from the `notifyCheckpointComplete` with `OperatorCoordinator`

approach?



About the discussion in FLINK-21133 and how it relates to FLIP-147. You are

right Yun gao, that in case of `stop-with-savepoint --drain` the whole job

finishes, while in FLIP-147 some of the sources can work for arbitrarily

long time after others have already finished. However from the runtime

perspective, especially how to close a Task and it's operators, there is no

difference between those two cases. In both cases, we need to end input,

shut down processing time timers and commit external side effects (two

phase commit support) before exiting the task. That's how the discussion

about the behaviour of "stop-with-savepoint" was connected with FLIP-147.



Currently on master, "stop-with-savepoint --drain" drains/flushes buffered

records and deals correctly with timers, but all of that happens AFTER

savepoint was completed. So any records flushed from the operators during

endOfInput/close/shutting down processing timers are never committed to

external systems. This is exactly the same problem as the "two phase

commit" problem of FLIP-147, that should have the same solution and it

should be solved at the same time. For example if we go with the *very

undesirable* "one closed operator per one completed checkpoint/savepoint",

in both cases CheckpointCoordinator, Scheduler and Task would need to keep

the task alive and keep triggering checkpoints for that task, until all

operators in the operator chain are closed (one closed operator per one

completed checkpoint).



Piotrek







śr., 24 lut 2021 o 11:30 Yun Gao  napisał(a):



> Hi Till, Guowei,

>

> Very thanks for initiating the disucssion and the deep thoughts!

>

> For the notifyCheckpointComplete, I also agree we could try to avoid

> emitting new records in notifyCheckpointComplete via using

> OperatorCoordinator

> for new sink API. Besides, the hive sink might also need some

modification

> for it also emits records in notifyCheckpointComplete.

>

> For unifying the process of stopping with savepoint and finished due to

> all records

> are processed, I also agree with that unifying would always be better if

> we could achieve,

> but I'm still not fully catch up with the implementation: Based on the

> discussion in FLINK-21133,

> my understanding is that for stopping with savepoint, now we want to

first

> stop the source, then we

> trigger a savepoint, and after the source received

> notifyCheckpointComplete, the source would

> start emitting EndOfPartitionEvent to finish the job, am I correct ?

>

> For normal finish, a difference to me might be if we have multiple

> sources, we could not guarantee

> when the sources are to finish. We might have one source run one 1 minute

> and another one run for

> 1 hour. To unify with the process with stop with savepoint, we might need

> to hold the fast source until

> all the sources are finished? An coordinator would be introduced to count

> the number of sources

> runing and trigger the final savepoint / checkpoint. For the extreme

> cases, if we have both bounded and

> unbounded sources, we might only count how much bounded source are

> remaining ? And if all the bounded

> sources are finished we would trigger the special checkpoint. After all

> the bounded part of the graph are

> finished, the the remaining part could still do checkpoint and commit

data

> with FLIP-147.

>

> Best,

> Yun

>

>

>

>

> ------------------Original Mail ------------------

> Sender:Guowei Ma 

> Send Date:Wed Feb 24 17:35:36 2021

> Recipients:dev 

> CC:Arvid Heise 

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

> Hi, Till

>

> Thank you very much for your careful consideration

>

>

>

> *1. Emit records in `NotifyCheckpointComplete`.*

>

> Sorry for making you misunderstanding because of my expression. I just

>

> want to say the current interface does not prevent users from doing it.

>

> From the perspective of the new sink api, we might not depend on emitting

>

> records in `NotifyCheckpointComplete`, like using `OperatorCoordinator`

>

> instead.

>

>

>

>

>

> *2. What does the FLIP-147 guarantee?*I think initially this FLIP want to

>

> achieve two targets:

>

> 1. Tasks/Operators exit correctly (as you mentioned the lifecycle of a

>

> Task/StreamTask/StreamOperator.).

>

> 2. Continue to trigger checkpoint after some tasks for mixed jobs.

>

>

>

> I think the first thing is related to the discussion in FLINK-21133. If I

>

> understand correctly, in addition to supporting the tasks / operators to

>

> exit correctly, now we also want to unify the process of the tasks and

>

> operators for savepoint / finish.

>

> I think the second thing is orthogonal to the FLINK-21133 because there

are

>

> topologies that have both the bounded and unbounded input.

>

>

>

> *3. How to unify the operator exit process of FLIP-147 with

>

> stop-with-savepoint?*

>

> I am not very sure about how to do it yet. But if I understand the

>

> discussion in the jira correctly it needs to introduce some logic into

>

> `CheckpointCoordinator`, which responses for triggering “the unified

>

> operator exit process”. Am I correct?

>

>

>

> Best,

>

> Guowei

>

>

>

>

>

> On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann wrote:

>

>

>

> > Thanks for the explanation Yun and Guowei. I have to admit that I do

not

>

> > fully understand why this is strictly required but I think that we are

>

> > touching two very important aspects which might have far fetching

>

> > consequences for how Flink works:

>

> >

>

> > 1) Do we want to allow that multiple checkpoints are required to

>

> > materialize results?

>

> > 2) Do we want to allow to emit records in notifyCheckpointComplete?

>

> >

>

> > For 1) I am not sure whether this has been discussed within the

community

>

> > sufficiently. Requiring multiple checkpoints to materialize a result

>

> > because of multi level committers has the consequence that we increase

> the

>

> > latency from checkpoint interval to #levels * checkpoint interval.

>

> > Moreover, having to drain the pipeline in multiple steps, would break

the

>

> > stop-with-savepoint --drain because which savepoint do you report to

the

>

> > user?

>

> >

>

> > For 2) allowing to send records after the final

notifyCheckpointComplete

>

> > will effectively mean that we need to shut down a topology in multiple

>

> > steps (in the worst case one operator per checkpoint). This would be a

>

> > strong argument for not allowing this to me. The fact that users can

send

>

> > records after the notifyCheckpointComplete is more by accident than by

>

> > design. I think we should make this a very deliberate decision and in

> doubt

>

> > I would be in favour of a more restrictive model unless there is a very

>

> > good reason why this should be supported.

>

> >

>

> > Taking also the discussion in FLINK-21133 [1] into account, it seems to

> me

>

> > that we haven't really understood what kind of guarantees we want to

give

>

> > to our users and how the final checkpoint should exactly work. I

> understand

>

> > that this is not included in the first scope of FLIP-147 but I think

this

>

> > is so important that we should figure this out asap. Also because the

> exact

>

> > shut down behaviour will have to be aligned with the lifecycle of a

>

> > Task/StreamTask/StreamOperator. And last but not least because other

>

> > features such as the new sink API start building upon a shut down model

>

> > which has not been fully understood/agreed upon.

>

> >

>

> > [1] https://issues.apache.org/jira/browse/FLINK-21133

>

> >

>

> > Cheers,

>

> > Till

>

> >

>

> > On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma wrote:

>

> >

>

> > > Thanks Yun for the detailed explanation.

>

> > > A simple supplementary explanation about the sink case: Maybe we

could

>

> > use

>

> > > `OperatorCoordinator` to avoid sending the element to the downstream

>

> > > operator.

>

> > > But I agree we could not limit the users not to emit records in the

>

> > > `notiyCheckpointComplete`.

>

> > >

>

> > > Best,

>

> > > Guowei

>

> > >

>

> > >

>

> > > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao

>

> > > wrote:

>

> > >

>

> > > > Hi all,

>

> > > >

>

> > > > I'd like to first detail the issue with emitting records in

>

> > > > notifyCheckpointComplete for context. For specific usage,

>

> > > > an example would be for sink, it might want to write some metadata

>

> > after

>

> > > > all the transactions are committed

>

> > > > (like write a marker file _SUCCESS to the output directory). This

> case

>

> > is

>

> > > > currently supported via the two level

>

> > > > committers of the new sink API: when received endOfInput(), the

>

> > Committer

>

> > > > wait for another checkpoint to

>

> > > > commits all the pending transactions and emit the list of files to

> the

>

> > > > GlobalCommitter. The GlobalCommitter

>

> > > > would wait for another checkpoint to also write the metadata with

2pc

>

> > > > (Although sometimes 2pc is not needed

>

> > > > for writing metadata, it should be only an optimization and still

>

> > > requires

>

> > > > the Committer do commit before

>

> > > > notifying the global Committer. Also another note is

GlobalCommitter

> is

>

> > > > also added for some other cases

>

> > > > like some sinks want an commiter with dop = 1, like IceBergSink).

>

> > > >

>

> > > > However, a more general issue to me is that currently we do not

limit

>

> > > > users to not emit records in

>

> > > > notifyCheckpointComplete in the API level. The sink case could be

>

> > viewed

>

> > > > as a special case, but in addition

>

> > > > to this one, logically users could also implement their own cases

> that

>

> > > > emits records in notifyCheckpointComplete.

>

> > > >

>

> > > > Best,

>

> > > > Yun

>

> > > >

>

> > > > ------------------Original Mail ------------------

>

> > > > Sender:Arvid Heise

>

> > > > Send Date:Fri Feb 12 20:46:04 2021

>

> > > > Recipients:dev

>

> > > > CC:Yun Gao

>

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

>

> > Finished

>

> > > > Hi Piotr,

>

> > > >

>

> > > >

>

> > > >

>

> > > > Thank you for raising your concern. Unfortunately, I do not have a

>

> > better

>

> > > >

>

> > > > idea than doing closing of operators intermittently with

checkpoints

> (=

>

> > > >

>

> > > > multiple last checkpoints).

>

> > > >

>

> > > >

>

> > > >

>

> > > > However, two ideas on how to improve the overall user experience:

>

> > > >

>

> > > > 1. If an operator is not relying on notifyCheckpointComplete, we

can

>

> > > close

>

> > > >

>

> > > > it faster (without waiting for a checkpoint). In general, I'd

assume

>

> > that

>

> > > >

>

> > > > almost all non-sinks behave that way.

>

> > > >

>

> > > > 2. We may increase the checkpointing frequency for the last

>

> > checkpoints.

>

> > > We

>

> > > >

>

> > > > need to avoid overloading checkpoint storages and task managers,

but

> I

>

> > > >

>

> > > > assume the more operators are closed, the lower the checkpointing

>

> > > interval

>

> > > >

>

> > > > can be.

>

> > > >

>

> > > >

>

> > > >

>

> > > > For 1, I'd propose to add (name TBD):

>

> > > >

>

> > > >

>

> > > >

>

> > > > default boolean StreamOperator#requiresFinalCheckpoint() {

>

> > > >

>

> > > > return true;

>

> > > >

>

> > > > }

>

> > > >

>

> > > >

>

> > > >

>

> > > > This means all operators are conservatively (=slowly) closed. For

> most

>

> > > >

>

> > > > operators, we can then define their behavior by overriding in

>

> > > >

>

> > > > AbstractUdfStreamOperator

>

> > > >

>

> > > >

>

> > > >

>

> > > > @Override

>

> > > >

>

> > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {

>

> > > >

>

> > > > return userFunction instanceof CheckpointListener;

>

> > > >

>

> > > > }

>

> > > >

>

> > > >

>

> > > >

>

> > > > This idea can be further refined in also adding

> requiresFinalCheckpoint

>

> > > to

>

> > > >

>

> > > > CheckpointListener to exclude all operators with UDFs that

implement

>

> > > >

>

> > > > CheckpointListener but do not need it for 2pc.

>

> > > >

>

> > > >

>

> > > >

>

> > > > @Override

>

> > > >

>

> > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {

>

> > > >

>

> > > > return userFunction instanceof CheckpointListener &&

>

> > > >

>

> > > > ((CheckpointListener) userFunction).requiresFinalCheckpoint();

>

> > > >

>

> > > > }

>

> > > >

>

> > > >

>

> > > >

>

> > > > That approach would also work for statebackends/snapshot strategies

>

> > that

>

> > > >

>

> > > > require some 2pc.

>

> > > >

>

> > > >

>

> > > >

>

> > > > If we can contain it to the @PublicEvolving StreamOperator, it

would

> be

>

> > > >

>

> > > > better of course.

>

> > > >

>

> > > >

>

> > > >

>

> > > > Best,

>

> > > >

>

> > > >

>

> > > >

>

> > > > Arvid

>

> > > >

>

> > > >

>

> > > >

>

> > > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski

>

> > > >

>

> > > > wrote:

>

> > > >

>

> > > >

>

> > > >

>

> > > > > Hey,

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > I would like to raise a concern about implementation of the final

>

> > > >

>

> > > > > checkpoints taking into account operators/functions that are

>

> > > implementing

>

> > > >

>

> > > > > two phase commit (2pc) protocol for exactly-once processing with

> some

>

> > > >

>

> > > > > external state (kept outside of the Flink). Primarily

exactly-once

>

> > > sinks.

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > First of all, as I understand it, this is not planned in the

first

>

> > > > version

>

> > > >

>

> > > > > of this FLIP. I'm fine with that, however I would strongly

> emphasize

>

> > > this

>

> > > >

>

> > > > > in every place we will be mentioning FLIP-147 efforts. This is

>

> > because

>

> > > > me,

>

> > > >

>

> > > > > as a user, upon hearing "Flink supports checkpointing with

bounded

>

> > > > inputs"

>

> > > >

>

> > > > > I would expect 2pc to work properly and to commit the external

side

>

> > > > effects

>

> > > >

>

> > > > > upon finishing. As it is now, I (as a user) would be surprised

> with a

>

> > > >

>

> > > > > silent data loss (of not committed trailing data). This is just a

>

> > > remark,

>

> > > >

>

> > > > > that we need to attach this warning to every blog

>

> > > post/documentation/user

>

> > > >

>

> > > > > mailing list response related to "Support Checkpoints After Tasks

>

> > > >

>

> > > > > Finished". Also I would suggest to prioritize the follow up of

>

> > > supporting

>

> > > >

>

> > > > > 2pc.

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > Secondly, I think we are missing how difficult and problematic

will

>

> > be

>

> > > > 2pc

>

> > > >

>

> > > > > support with the final checkpoint.

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > For starters, keep in mind that currently 2pc can be implemented

by

>

> > > users

>

> > > >

>

> > > > > using both `@Public` APIs as functions and `@PublicEvolving`

>

> > operators

>

> > > in

>

> > > >

>

> > > > > any place in the job graph. It's not limited to only the sinks.

For

>

> > > >

>

> > > > > example users could easily implement the `AsynFunction` (for

>

> > > >

>

> > > > > `AsyncWaitOperator`) that is using 2pc based on the

>

> > > `CheckpointListener`

>

> > > >

>

> > > > > interface. I'm not saying it's common, probably just a tiny

> minority

>

> > of

>

> > > >

>

> > > > > users are doing that (if any at all), but nevertheless that's

>

> > possible

>

> > > > and

>

> > > >

>

> > > > > currently (implicitly?) supported in Flink.

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > Next complication is the support of bounded streams

>

> > (`BoundedOneInput`

>

> > > or

>

> > > >

>

> > > > > `BoundedMultiInput` interfaces) and the closing/shutdown

procedure

> of

>

> > > the

>

> > > >

>

> > > > > operators. Currently it works as follows:

>

> > > >

>

> > > > > 0. Task receives EndOfPartitionEvent (or source finishes)

>

> > > >

>

> > > > > 1. `endOfInput` is called on the first operator in the chain

>

> > > >

>

> > > > > 2. We quiesce the processing timers

>

> > > >

>

> > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for

> the

>

> > > > first

>

> > > >

>

> > > > > operator, so no new timers will be triggered

>

> > > >

>

> > > > > 3. We wait for the already fired timers to finish executing

> (spinning

>

> > > >

>

> > > > > mailbox loop)

>

> > > >

>

> > > > > 4. We are closing the first operator

>

> > > >

>

> > > > > 5. We go to the next (second) operator in the chain and repeat

the

>

> > > steps

>

> > > > 1.

>

> > > >

>

> > > > > to 5.

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > This is because operators can emit data after processing

>

> > `endOfInput`,

>

> > > > from

>

> > > >

>

> > > > > timers, async mailbox actions and inside the `close` method

itself.

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > Now the problem is to support the final checkpoint with 2pc, we

> need

>

> > > >

>

> > > > > trigger `snapshotState` and `notifyCheckpointComplete` call at

the

>

> > very

>

> > > >

>

> > > > > least only after `endOfInput` call on the operator. Probably the

> best

>

> > > > place

>

> > > >

>

> > > > > would be in between steps 3. and 4. However that means, we would

be

>

> > > > forced

>

> > > >

>

> > > > > to wait for steps 1. to 3. to finish, then wait for a next

> checkpoint

>

> > > to

>

> > > >

>

> > > > > trigger AND complete, before finally closing the head operator,

and

>

> > > only

>

> > > >

>

> > > > > then we can start closing the next operator in the chain:

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > 0. Task receives EndOfPartitionEvent (or source finishes)

>

> > > >

>

> > > > > 1. `endOfInput` is called on the first operator in the chain

>

> > > >

>

> > > > > 2. We quiesce the processing timers

>

> > > >

>

> > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for

> the

>

> > > > first

>

> > > >

>

> > > > > operator, so no new timers will be triggered

>

> > > >

>

> > > > > 3. We wait for the already fired timers to finish executing

> (spinning

>

> > > >

>

> > > > > mailbox loop)

>

> > > >

>

> > > > > *3b. We wait for one more checkpoint to trigger and for the

>

> > > >

>

> > > > > `notifyCheckpointComplete` RPC.*

>

> > > >

>

> > > > > 4. We are closing the first operator

>

> > > >

>

> > > > > 5. We go to the next (second) operator in the chain and repeat

the

>

> > > steps

>

> > > > 1.

>

> > > >

>

> > > > > to 5.

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > That means, we can close one operator per successful checkpoint.

To

>

> > > close

>

> > > >

>

> > > > > 10 operators, we would need 10 successful checkpoints.

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > I was thinking about different approaches to this problem, and I

>

> > > couldn't

>

> > > >

>

> > > > > find any viable ones. All I could think of would break the

current

>

> > > >

>

> > > > > `@Public` API and/or would be ugly/confusing for the users.

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > For example a relatively simple solution, to introduce a

`preClose`

>

> > or

>

> > > >

>

> > > > > `flush` method to the operators, with a contract that after

>

> > > >

>

> > > > > `flush`, operators would be forbidden from emitting more records,

> so

>

> > > that

>

> > > >

>

> > > > > we can replace step 4. with this `flush` call, and then having a

>

> > single

>

> > > >

>

> > > > > checkpoint to finish 2pc for all of the operators inside the

chain,

>

> > > > doesn't

>

> > > >

>

> > > > > work. Sheer fact of adding this `flush` method and changing the

>

> > > contract

>

> > > >

>

> > > > > would break the current API and Yun Gao has pointed out to me,

that

>

> > we

>

> > > >

>

> > > > > either already support, or want to support operators that are

>

> > emitting

>

> > > >

>

> > > > > records from within the `notifyCheckpointComplete` call:

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > > Yun Gao:

>

> > > >

>

> > > > > > like with the new sink api there might be writer -> committer

->

>

> > > global

>

> > > >

>

> > > > > committer, the committer would need to wait for the last

checkpoint

>

> > to

>

> > > >

>

> > > > > commit

>

> > > >

>

> > > > > > the last piece of data, and after that it also need to emit the

>

> > list

>

> > > of

>

> > > >

>

> > > > > transactions get committed to global committer to do some

>

> > finalization

>

> > > >

>

> > > > > logic.

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > So it wouldn't solve the problem (at least not fully).

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > I don't know if anyone has any better ideas how to solve this

>

> > problem?

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > Piotrek

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > pt., 15 sty 2021 o 14:57 Yun Gao

>

> > > >

>

> > > > > napisał(a):

>

> > > >

>

> > > > >

>

> > > >

>

> > > > > > Hi Aljoscha,

>

> > > >

>

> > > > > >

>

> > > >

>

> > > > > > I think so since we seems to do not have other divergence and

new

>

> > > >

>

> > > > > > objections now. I'll open the vote then. Very thanks!

>

> > > >

>

> > > > > >

>

> > > >

>

> > > > > > Best,

>

> > > >

>

> > > > > > Yun

>

> > > >

>

> > > > > >

>

> > > >

>

> > > > > >

>

> > > >

>

> > > > > >

> ------------------------------------------------------------------

>

> > > >

>

> > > > > > From:Aljoscha Krettek

>

> > > >

>

> > > > > > Send Time:2021 Jan. 15 (Fri.) 21:24

>

> > > >

>

> > > > > > To:dev

>

> > > >

>

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

>

> > > > Finished

>

> > > >

>

> > > > > >

>

> > > >

>

> > > > > > Thanks for the summary! I think we can now move towards a

[VOTE]

>

> > > > thread,

>

> > > >

>

> > > > > > right?

>

> > > >

>

> > > > > >

>

> > > >

>

> > > > > > On 2021/01/15 13:43, Yun Gao wrote:

>

> > > >

>

> > > > > > >1) For the problem that the "new" root task coincidently

> finished

>

> > > >

>

> > > > > > >before getting triggered successfully, we have listed two

> options

>

> > in

>

> > > >

>

> > > > > > >the FLIP-147[1], for the first version, now we are not tend to

> go

>

> > > with

>

> > > >

>

> > > > > > >the first option that JM would re-compute and re-trigger new

>

> > sources

>

> > > >

>

> > > > > > >when it realized some tasks are not triggered successfully.

This

>

> > > > option

>

> > > >

>

> > > > > > >would avoid the complexity of adding new PRC and duplicating

> task

>

> > > >

>

> > > > > > >states, and in average case it would not cause too much

> overhead.

>

> > > >

>

> > > > > >

>

> > > >

>

> > > > > > You wrote "we are *not* tend to go with the first option", but

I

>

> > > think

>

> > > >

>

> > > > > > you meant wo write "we tend to *now* go with the first option",

>

> > > right?

>

> > > >

>

> > > > > > That's also how it is in the FLIP, I just wanted to clarify for

> the

>

> > > >

>

> > > > > > mailing list.

>

> > > >

>

> > > > > >

>

> > > >

>

> > > > > >

>

> > > >

>

> > > > >

>

> > > >

>

> > > >

>

> > >

>

> >

>

>


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

Posted by Kezhu Wang <ke...@gmail.com>.
Hi all, thanks for driving this and especially Piotr for re-active this
thread.

First, for `notifyCheckpointComplete`, I have strong preference towards
"shut down the dataflow
pipeline with one checkpoint in total", so I tend to option dropping "send
records" from
`notifyCheckpointComplete` for next level committing in pipeline, if we
ever support it. Without this,
we are unable to stop a pipeline manually with all results materialized.

Second, for shutdown unification of `stop-with-savepoint --drain` and
FLIP-147, I draft following
phase based on emerging proposals(mainly by Piotr and Till) in this thread
and FLINK-21133.

```java
// StreamTask termination phase
finish()(Call StreamOperator.finish in chaining order)
advanceToEndOfEventTime()(nop if for no-source inputs, this could also be
done in input processor or finish ?)
if (there-is-a-pending-terminate-savepoint) {
  triggerBarrierForDownStream();
  waitCheckpointComplete();
} else if (require-two-phase-commit-in-shutdown) {
  waitFinalCheckpoint();
  waitCheckpointComplete();
}
// TODO: Ignore/Cancel all checkpoints from this point(or hit to
coordinator in final checkpoint response ?).
close();
```

Branches in `if` could reside in different code paths, but the effect
should be similar to above.
The divergence reflects fact that "stop-with-savepoint --drain" is
triggered already while we need
to wait for final checkpoint in natural input exhausting.

With carefully chosen default functions, we will not break existing
interfaces.
* `StreamOperator.finish`
* `RichFunction.finish`

For existing codes:
1. "stop-with-savepoint" with no migration, it should behaves as before.
2. "require-two-phase-commit-in-shutdown" evaluates to false and no
migration, it is same as before.
3. "require-two-phase-commit-in-shutdown" evaluates to true and no
migration, records in `close` will
  lose.

For case#3:
A. No external materialization in `StreamOperator.close`: Nothing to lose.
B. External materialization only in `StreamOperator.close`: I don't think
Flink ever claimed that there is
   any guarantee for this situation.
C. External materialization in `notifyCheckpointComplete` from
`StreamOperator.close`: But due to fact that
   `notifyCheckpointComplete` was ignored after operator
closed(FLINK-16383), so there will be no external
   materialization to lose.

Then, we could recommend users to migrate possible “end-of-stream-flushing”
from “close" to “finish”.

Best,
Kezhu Wang


On February 24, 2021 at 23:47:36, Piotr Nowojski (piotr.nowojski@gmail.com)
wrote:

Thanks for the reponses Guowei and Yun,

Could you elaborate more/remind me, what does it mean to replace emitting
results from the `notifyCheckpointComplete` with `OperatorCoordinator`
approach?

About the discussion in FLINK-21133 and how it relates to FLIP-147. You are
right Yun gao, that in case of `stop-with-savepoint --drain` the whole job
finishes, while in FLIP-147 some of the sources can work for arbitrarily
long time after others have already finished. However from the runtime
perspective, especially how to close a Task and it's operators, there is no
difference between those two cases. In both cases, we need to end input,
shut down processing time timers and commit external side effects (two
phase commit support) before exiting the task. That's how the discussion
about the behaviour of "stop-with-savepoint" was connected with FLIP-147.

Currently on master, "stop-with-savepoint --drain" drains/flushes buffered
records and deals correctly with timers, but all of that happens AFTER
savepoint was completed. So any records flushed from the operators during
endOfInput/close/shutting down processing timers are never committed to
external systems. This is exactly the same problem as the "two phase
commit" problem of FLIP-147, that should have the same solution and it
should be solved at the same time. For example if we go with the *very
undesirable* "one closed operator per one completed checkpoint/savepoint",
in both cases CheckpointCoordinator, Scheduler and Task would need to keep
the task alive and keep triggering checkpoints for that task, until all
operators in the operator chain are closed (one closed operator per one
completed checkpoint).

Piotrek



śr., 24 lut 2021 o 11:30 Yun Gao <yu...@aliyun.com.invalid> napisał(a):

> Hi Till, Guowei,
>
> Very thanks for initiating the disucssion and the deep thoughts!
>
> For the notifyCheckpointComplete, I also agree we could try to avoid
> emitting new records in notifyCheckpointComplete via using
> OperatorCoordinator
> for new sink API. Besides, the hive sink might also need some
modification
> for it also emits records in notifyCheckpointComplete.
>
> For unifying the process of stopping with savepoint and finished due to
> all records
> are processed, I also agree with that unifying would always be better if
> we could achieve,
> but I'm still not fully catch up with the implementation: Based on the
> discussion in FLINK-21133,
> my understanding is that for stopping with savepoint, now we want to
first
> stop the source, then we
> trigger a savepoint, and after the source received
> notifyCheckpointComplete, the source would
> start emitting EndOfPartitionEvent to finish the job, am I correct ?
>
> For normal finish, a difference to me might be if we have multiple
> sources, we could not guarantee
> when the sources are to finish. We might have one source run one 1 minute
> and another one run for
> 1 hour. To unify with the process with stop with savepoint, we might need
> to hold the fast source until
> all the sources are finished? An coordinator would be introduced to count
> the number of sources
> runing and trigger the final savepoint / checkpoint. For the extreme
> cases, if we have both bounded and
> unbounded sources, we might only count how much bounded source are
> remaining ? And if all the bounded
> sources are finished we would trigger the special checkpoint. After all
> the bounded part of the graph are
> finished, the the remaining part could still do checkpoint and commit
data
> with FLIP-147.
>
> Best,
> Yun
>
>
>
>
> ------------------Original Mail ------------------
> Sender:Guowei Ma <gu...@gmail.com>
> Send Date:Wed Feb 24 17:35:36 2021
> Recipients:dev <de...@flink.apache.org>
> CC:Arvid Heise <ar...@apache.org>
> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
> Hi, Till
>
> Thank you very much for your careful consideration
>
>
>
> *1. Emit records in `NotifyCheckpointComplete`.*
>
> Sorry for making you misunderstanding because of my expression. I just
>
> want to say the current interface does not prevent users from doing it.
>
> From the perspective of the new sink api, we might not depend on emitting
>
> records in `NotifyCheckpointComplete`, like using `OperatorCoordinator`
>
> instead.
>
>
>
>
>
> *2. What does the FLIP-147 guarantee?*I think initially this FLIP want to
>
> achieve two targets:
>
> 1. Tasks/Operators exit correctly (as you mentioned the lifecycle of a
>
> Task/StreamTask/StreamOperator.).
>
> 2. Continue to trigger checkpoint after some tasks for mixed jobs.
>
>
>
> I think the first thing is related to the discussion in FLINK-21133. If I
>
> understand correctly, in addition to supporting the tasks / operators to
>
> exit correctly, now we also want to unify the process of the tasks and
>
> operators for savepoint / finish.
>
> I think the second thing is orthogonal to the FLINK-21133 because there
are
>
> topologies that have both the bounded and unbounded input.
>
>
>
> *3. How to unify the operator exit process of FLIP-147 with
>
> stop-with-savepoint?*
>
> I am not very sure about how to do it yet. But if I understand the
>
> discussion in the jira correctly it needs to introduce some logic into
>
> `CheckpointCoordinator`, which responses for triggering “the unified
>
> operator exit process”. Am I correct?
>
>
>
> Best,
>
> Guowei
>
>
>
>
>
> On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann wrote:
>
>
>
> > Thanks for the explanation Yun and Guowei. I have to admit that I do
not
>
> > fully understand why this is strictly required but I think that we are
>
> > touching two very important aspects which might have far fetching
>
> > consequences for how Flink works:
>
> >
>
> > 1) Do we want to allow that multiple checkpoints are required to
>
> > materialize results?
>
> > 2) Do we want to allow to emit records in notifyCheckpointComplete?
>
> >
>
> > For 1) I am not sure whether this has been discussed within the
community
>
> > sufficiently. Requiring multiple checkpoints to materialize a result
>
> > because of multi level committers has the consequence that we increase
> the
>
> > latency from checkpoint interval to #levels * checkpoint interval.
>
> > Moreover, having to drain the pipeline in multiple steps, would break
the
>
> > stop-with-savepoint --drain because which savepoint do you report to
the
>
> > user?
>
> >
>
> > For 2) allowing to send records after the final
notifyCheckpointComplete
>
> > will effectively mean that we need to shut down a topology in multiple
>
> > steps (in the worst case one operator per checkpoint). This would be a
>
> > strong argument for not allowing this to me. The fact that users can
send
>
> > records after the notifyCheckpointComplete is more by accident than by
>
> > design. I think we should make this a very deliberate decision and in
> doubt
>
> > I would be in favour of a more restrictive model unless there is a very
>
> > good reason why this should be supported.
>
> >
>
> > Taking also the discussion in FLINK-21133 [1] into account, it seems to
> me
>
> > that we haven't really understood what kind of guarantees we want to
give
>
> > to our users and how the final checkpoint should exactly work. I
> understand
>
> > that this is not included in the first scope of FLIP-147 but I think
this
>
> > is so important that we should figure this out asap. Also because the
> exact
>
> > shut down behaviour will have to be aligned with the lifecycle of a
>
> > Task/StreamTask/StreamOperator. And last but not least because other
>
> > features such as the new sink API start building upon a shut down model
>
> > which has not been fully understood/agreed upon.
>
> >
>
> > [1] https://issues.apache.org/jira/browse/FLINK-21133
>
> >
>
> > Cheers,
>
> > Till
>
> >
>
> > On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma wrote:
>
> >
>
> > > Thanks Yun for the detailed explanation.
>
> > > A simple supplementary explanation about the sink case: Maybe we
could
>
> > use
>
> > > `OperatorCoordinator` to avoid sending the element to the downstream
>
> > > operator.
>
> > > But I agree we could not limit the users not to emit records in the
>
> > > `notiyCheckpointComplete`.
>
> > >
>
> > > Best,
>
> > > Guowei
>
> > >
>
> > >
>
> > > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao
>
> > > wrote:
>
> > >
>
> > > > Hi all,
>
> > > >
>
> > > > I'd like to first detail the issue with emitting records in
>
> > > > notifyCheckpointComplete for context. For specific usage,
>
> > > > an example would be for sink, it might want to write some metadata
>
> > after
>
> > > > all the transactions are committed
>
> > > > (like write a marker file _SUCCESS to the output directory). This
> case
>
> > is
>
> > > > currently supported via the two level
>
> > > > committers of the new sink API: when received endOfInput(), the
>
> > Committer
>
> > > > wait for another checkpoint to
>
> > > > commits all the pending transactions and emit the list of files to
> the
>
> > > > GlobalCommitter. The GlobalCommitter
>
> > > > would wait for another checkpoint to also write the metadata with
2pc
>
> > > > (Although sometimes 2pc is not needed
>
> > > > for writing metadata, it should be only an optimization and still
>
> > > requires
>
> > > > the Committer do commit before
>
> > > > notifying the global Committer. Also another note is
GlobalCommitter
> is
>
> > > > also added for some other cases
>
> > > > like some sinks want an commiter with dop = 1, like IceBergSink).
>
> > > >
>
> > > > However, a more general issue to me is that currently we do not
limit
>
> > > > users to not emit records in
>
> > > > notifyCheckpointComplete in the API level. The sink case could be
>
> > viewed
>
> > > > as a special case, but in addition
>
> > > > to this one, logically users could also implement their own cases
> that
>
> > > > emits records in notifyCheckpointComplete.
>
> > > >
>
> > > > Best,
>
> > > > Yun
>
> > > >
>
> > > > ------------------Original Mail ------------------
>
> > > > Sender:Arvid Heise
>
> > > > Send Date:Fri Feb 12 20:46:04 2021
>
> > > > Recipients:dev
>
> > > > CC:Yun Gao
>
> > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
>
> > Finished
>
> > > > Hi Piotr,
>
> > > >
>
> > > >
>
> > > >
>
> > > > Thank you for raising your concern. Unfortunately, I do not have a
>
> > better
>
> > > >
>
> > > > idea than doing closing of operators intermittently with
checkpoints
> (=
>
> > > >
>
> > > > multiple last checkpoints).
>
> > > >
>
> > > >
>
> > > >
>
> > > > However, two ideas on how to improve the overall user experience:
>
> > > >
>
> > > > 1. If an operator is not relying on notifyCheckpointComplete, we
can
>
> > > close
>
> > > >
>
> > > > it faster (without waiting for a checkpoint). In general, I'd
assume
>
> > that
>
> > > >
>
> > > > almost all non-sinks behave that way.
>
> > > >
>
> > > > 2. We may increase the checkpointing frequency for the last
>
> > checkpoints.
>
> > > We
>
> > > >
>
> > > > need to avoid overloading checkpoint storages and task managers,
but
> I
>
> > > >
>
> > > > assume the more operators are closed, the lower the checkpointing
>
> > > interval
>
> > > >
>
> > > > can be.
>
> > > >
>
> > > >
>
> > > >
>
> > > > For 1, I'd propose to add (name TBD):
>
> > > >
>
> > > >
>
> > > >
>
> > > > default boolean StreamOperator#requiresFinalCheckpoint() {
>
> > > >
>
> > > > return true;
>
> > > >
>
> > > > }
>
> > > >
>
> > > >
>
> > > >
>
> > > > This means all operators are conservatively (=slowly) closed. For
> most
>
> > > >
>
> > > > operators, we can then define their behavior by overriding in
>
> > > >
>
> > > > AbstractUdfStreamOperator
>
> > > >
>
> > > >
>
> > > >
>
> > > > @Override
>
> > > >
>
> > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
>
> > > >
>
> > > > return userFunction instanceof CheckpointListener;
>
> > > >
>
> > > > }
>
> > > >
>
> > > >
>
> > > >
>
> > > > This idea can be further refined in also adding
> requiresFinalCheckpoint
>
> > > to
>
> > > >
>
> > > > CheckpointListener to exclude all operators with UDFs that
implement
>
> > > >
>
> > > > CheckpointListener but do not need it for 2pc.
>
> > > >
>
> > > >
>
> > > >
>
> > > > @Override
>
> > > >
>
> > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
>
> > > >
>
> > > > return userFunction instanceof CheckpointListener &&
>
> > > >
>
> > > > ((CheckpointListener) userFunction).requiresFinalCheckpoint();
>
> > > >
>
> > > > }
>
> > > >
>
> > > >
>
> > > >
>
> > > > That approach would also work for statebackends/snapshot strategies
>
> > that
>
> > > >
>
> > > > require some 2pc.
>
> > > >
>
> > > >
>
> > > >
>
> > > > If we can contain it to the @PublicEvolving StreamOperator, it
would
> be
>
> > > >
>
> > > > better of course.
>
> > > >
>
> > > >
>
> > > >
>
> > > > Best,
>
> > > >
>
> > > >
>
> > > >
>
> > > > Arvid
>
> > > >
>
> > > >
>
> > > >
>
> > > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
>
> > > >
>
> > > > wrote:
>
> > > >
>
> > > >
>
> > > >
>
> > > > > Hey,
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > I would like to raise a concern about implementation of the final
>
> > > >
>
> > > > > checkpoints taking into account operators/functions that are
>
> > > implementing
>
> > > >
>
> > > > > two phase commit (2pc) protocol for exactly-once processing with
> some
>
> > > >
>
> > > > > external state (kept outside of the Flink). Primarily
exactly-once
>
> > > sinks.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > First of all, as I understand it, this is not planned in the
first
>
> > > > version
>
> > > >
>
> > > > > of this FLIP. I'm fine with that, however I would strongly
> emphasize
>
> > > this
>
> > > >
>
> > > > > in every place we will be mentioning FLIP-147 efforts. This is
>
> > because
>
> > > > me,
>
> > > >
>
> > > > > as a user, upon hearing "Flink supports checkpointing with
bounded
>
> > > > inputs"
>
> > > >
>
> > > > > I would expect 2pc to work properly and to commit the external
side
>
> > > > effects
>
> > > >
>
> > > > > upon finishing. As it is now, I (as a user) would be surprised
> with a
>
> > > >
>
> > > > > silent data loss (of not committed trailing data). This is just a
>
> > > remark,
>
> > > >
>
> > > > > that we need to attach this warning to every blog
>
> > > post/documentation/user
>
> > > >
>
> > > > > mailing list response related to "Support Checkpoints After Tasks
>
> > > >
>
> > > > > Finished". Also I would suggest to prioritize the follow up of
>
> > > supporting
>
> > > >
>
> > > > > 2pc.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > Secondly, I think we are missing how difficult and problematic
will
>
> > be
>
> > > > 2pc
>
> > > >
>
> > > > > support with the final checkpoint.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > For starters, keep in mind that currently 2pc can be implemented
by
>
> > > users
>
> > > >
>
> > > > > using both `@Public` APIs as functions and `@PublicEvolving`
>
> > operators
>
> > > in
>
> > > >
>
> > > > > any place in the job graph. It's not limited to only the sinks.
For
>
> > > >
>
> > > > > example users could easily implement the `AsynFunction` (for
>
> > > >
>
> > > > > `AsyncWaitOperator`) that is using 2pc based on the
>
> > > `CheckpointListener`
>
> > > >
>
> > > > > interface. I'm not saying it's common, probably just a tiny
> minority
>
> > of
>
> > > >
>
> > > > > users are doing that (if any at all), but nevertheless that's
>
> > possible
>
> > > > and
>
> > > >
>
> > > > > currently (implicitly?) supported in Flink.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > Next complication is the support of bounded streams
>
> > (`BoundedOneInput`
>
> > > or
>
> > > >
>
> > > > > `BoundedMultiInput` interfaces) and the closing/shutdown
procedure
> of
>
> > > the
>
> > > >
>
> > > > > operators. Currently it works as follows:
>
> > > >
>
> > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
>
> > > >
>
> > > > > 1. `endOfInput` is called on the first operator in the chain
>
> > > >
>
> > > > > 2. We quiesce the processing timers
>
> > > >
>
> > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for
> the
>
> > > > first
>
> > > >
>
> > > > > operator, so no new timers will be triggered
>
> > > >
>
> > > > > 3. We wait for the already fired timers to finish executing
> (spinning
>
> > > >
>
> > > > > mailbox loop)
>
> > > >
>
> > > > > 4. We are closing the first operator
>
> > > >
>
> > > > > 5. We go to the next (second) operator in the chain and repeat
the
>
> > > steps
>
> > > > 1.
>
> > > >
>
> > > > > to 5.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > This is because operators can emit data after processing
>
> > `endOfInput`,
>
> > > > from
>
> > > >
>
> > > > > timers, async mailbox actions and inside the `close` method
itself.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > Now the problem is to support the final checkpoint with 2pc, we
> need
>
> > > >
>
> > > > > trigger `snapshotState` and `notifyCheckpointComplete` call at
the
>
> > very
>
> > > >
>
> > > > > least only after `endOfInput` call on the operator. Probably the
> best
>
> > > > place
>
> > > >
>
> > > > > would be in between steps 3. and 4. However that means, we would
be
>
> > > > forced
>
> > > >
>
> > > > > to wait for steps 1. to 3. to finish, then wait for a next
> checkpoint
>
> > > to
>
> > > >
>
> > > > > trigger AND complete, before finally closing the head operator,
and
>
> > > only
>
> > > >
>
> > > > > then we can start closing the next operator in the chain:
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
>
> > > >
>
> > > > > 1. `endOfInput` is called on the first operator in the chain
>
> > > >
>
> > > > > 2. We quiesce the processing timers
>
> > > >
>
> > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for
> the
>
> > > > first
>
> > > >
>
> > > > > operator, so no new timers will be triggered
>
> > > >
>
> > > > > 3. We wait for the already fired timers to finish executing
> (spinning
>
> > > >
>
> > > > > mailbox loop)
>
> > > >
>
> > > > > *3b. We wait for one more checkpoint to trigger and for the
>
> > > >
>
> > > > > `notifyCheckpointComplete` RPC.*
>
> > > >
>
> > > > > 4. We are closing the first operator
>
> > > >
>
> > > > > 5. We go to the next (second) operator in the chain and repeat
the
>
> > > steps
>
> > > > 1.
>
> > > >
>
> > > > > to 5.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > That means, we can close one operator per successful checkpoint.
To
>
> > > close
>
> > > >
>
> > > > > 10 operators, we would need 10 successful checkpoints.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > I was thinking about different approaches to this problem, and I
>
> > > couldn't
>
> > > >
>
> > > > > find any viable ones. All I could think of would break the
current
>
> > > >
>
> > > > > `@Public` API and/or would be ugly/confusing for the users.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > For example a relatively simple solution, to introduce a
`preClose`
>
> > or
>
> > > >
>
> > > > > `flush` method to the operators, with a contract that after
>
> > > >
>
> > > > > `flush`, operators would be forbidden from emitting more records,
> so
>
> > > that
>
> > > >
>
> > > > > we can replace step 4. with this `flush` call, and then having a
>
> > single
>
> > > >
>
> > > > > checkpoint to finish 2pc for all of the operators inside the
chain,
>
> > > > doesn't
>
> > > >
>
> > > > > work. Sheer fact of adding this `flush` method and changing the
>
> > > contract
>
> > > >
>
> > > > > would break the current API and Yun Gao has pointed out to me,
that
>
> > we
>
> > > >
>
> > > > > either already support, or want to support operators that are
>
> > emitting
>
> > > >
>
> > > > > records from within the `notifyCheckpointComplete` call:
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > > Yun Gao:
>
> > > >
>
> > > > > > like with the new sink api there might be writer -> committer
->
>
> > > global
>
> > > >
>
> > > > > committer, the committer would need to wait for the last
checkpoint
>
> > to
>
> > > >
>
> > > > > commit
>
> > > >
>
> > > > > > the last piece of data, and after that it also need to emit the
>
> > list
>
> > > of
>
> > > >
>
> > > > > transactions get committed to global committer to do some
>
> > finalization
>
> > > >
>
> > > > > logic.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > So it wouldn't solve the problem (at least not fully).
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > I don't know if anyone has any better ideas how to solve this
>
> > problem?
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > Piotrek
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > pt., 15 sty 2021 o 14:57 Yun Gao
>
> > > >
>
> > > > > napisał(a):
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > > Hi Aljoscha,
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > > I think so since we seems to do not have other divergence and
new
>
> > > >
>
> > > > > > objections now. I'll open the vote then. Very thanks!
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > > Best,
>
> > > >
>
> > > > > > Yun
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > >
> ------------------------------------------------------------------
>
> > > >
>
> > > > > > From:Aljoscha Krettek
>
> > > >
>
> > > > > > Send Time:2021 Jan. 15 (Fri.) 21:24
>
> > > >
>
> > > > > > To:dev
>
> > > >
>
> > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
>
> > > > Finished
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > > Thanks for the summary! I think we can now move towards a
[VOTE]
>
> > > > thread,
>
> > > >
>
> > > > > > right?
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > > On 2021/01/15 13:43, Yun Gao wrote:
>
> > > >
>
> > > > > > >1) For the problem that the "new" root task coincidently
> finished
>
> > > >
>
> > > > > > >before getting triggered successfully, we have listed two
> options
>
> > in
>
> > > >
>
> > > > > > >the FLIP-147[1], for the first version, now we are not tend to
> go
>
> > > with
>
> > > >
>
> > > > > > >the first option that JM would re-compute and re-trigger new
>
> > sources
>
> > > >
>
> > > > > > >when it realized some tasks are not triggered successfully.
This
>
> > > > option
>
> > > >
>
> > > > > > >would avoid the complexity of adding new PRC and duplicating
> task
>
> > > >
>
> > > > > > >states, and in average case it would not cause too much
> overhead.
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > > You wrote "we are *not* tend to go with the first option", but
I
>
> > > think
>
> > > >
>
> > > > > > you meant wo write "we tend to *now* go with the first option",
>
> > > right?
>
> > > >
>
> > > > > > That's also how it is in the FLIP, I just wanted to clarify for
> the
>
> > > >
>
> > > > > > mailing list.
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > >
>
> > > >
>
> > > >
>
> > >
>
> >
>
>

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

Posted by Piotr Nowojski <pi...@gmail.com>.
Thanks for the reponses Guowei and Yun,

Could you elaborate more/remind me, what does it mean to replace emitting
results from the `notifyCheckpointComplete` with `OperatorCoordinator`
approach?

About the discussion in FLINK-21133 and how it relates to FLIP-147. You are
right Yun gao, that in case of `stop-with-savepoint --drain` the whole job
finishes, while in FLIP-147 some of the sources can work for arbitrarily
long time after others have already finished. However from the runtime
perspective, especially how to close a Task and it's operators, there is no
difference between those two cases. In both cases, we need to end input,
shut down processing time timers and commit external side effects (two
phase commit support) before exiting the task. That's how the discussion
about the behaviour of "stop-with-savepoint" was connected with FLIP-147.

Currently on master, "stop-with-savepoint --drain" drains/flushes buffered
records and deals correctly with timers, but all of that happens AFTER
savepoint was completed. So any records flushed from the operators during
endOfInput/close/shutting down processing timers are never committed to
external systems. This is exactly the same problem as the "two phase
commit" problem of FLIP-147, that should have the same solution and it
should be solved at the same time. For example if we go with the *very
undesirable* "one closed operator per one completed checkpoint/savepoint",
in both cases CheckpointCoordinator, Scheduler and Task would need to keep
the task alive and keep triggering checkpoints for that task, until all
operators in the operator chain are closed (one closed operator per one
completed checkpoint).

Piotrek



śr., 24 lut 2021 o 11:30 Yun Gao <yu...@aliyun.com.invalid> napisał(a):

> Hi Till, Guowei,
>
> Very thanks for initiating the disucssion and the deep thoughts!
>
> For the notifyCheckpointComplete, I also agree we could try to avoid
> emitting new records in notifyCheckpointComplete via using
> OperatorCoordinator
> for new sink API. Besides, the hive sink might also need some modification
> for it also emits records in notifyCheckpointComplete.
>
> For unifying the process of stopping with savepoint and finished due to
> all records
> are processed, I also agree with that unifying would always be better if
> we could achieve,
> but I'm still not fully catch up with the implementation: Based on the
> discussion in FLINK-21133,
> my understanding is that for stopping with savepoint, now we want to first
> stop the source, then we
> trigger a savepoint, and after the source received
> notifyCheckpointComplete, the source would
> start emitting EndOfPartitionEvent to finish the job, am I correct ?
>
> For normal finish, a difference to me might be if we have multiple
> sources, we could not guarantee
> when the sources are to finish. We might have one source run one 1 minute
> and another one run for
> 1 hour. To unify with the process with stop with savepoint, we might need
> to hold the fast source until
> all the sources are finished? An coordinator would be introduced to count
> the number of sources
> runing and trigger the final savepoint / checkpoint. For the extreme
> cases, if we have both bounded and
> unbounded sources, we might only count how much bounded source are
> remaining ? And if all the bounded
> sources are finished we would trigger the special checkpoint. After all
> the bounded part of the graph are
> finished, the the remaining part could still do checkpoint and commit data
> with FLIP-147.
>
> Best,
> Yun
>
>
>
>
>  ------------------Original Mail ------------------
> Sender:Guowei Ma <gu...@gmail.com>
> Send Date:Wed Feb 24 17:35:36 2021
> Recipients:dev <de...@flink.apache.org>
> CC:Arvid Heise <ar...@apache.org>
> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
> Hi, Till
>
> Thank you very much for your careful consideration
>
>
>
> *1. Emit records in `NotifyCheckpointComplete`.*
>
>  Sorry for making you misunderstanding because of my expression. I just
>
> want to say the current interface does not prevent users from doing it.
>
> From the perspective of the new sink api, we might not depend on emitting
>
> records in `NotifyCheckpointComplete`, like using `OperatorCoordinator`
>
> instead.
>
>
>
>
>
> *2. What does the FLIP-147 guarantee?*I think initially this FLIP want to
>
> achieve two targets:
>
> 1. Tasks/Operators exit correctly (as you mentioned the lifecycle of a
>
> Task/StreamTask/StreamOperator.).
>
> 2. Continue to trigger checkpoint after some tasks for mixed jobs.
>
>
>
> I think the first thing is related to the discussion in FLINK-21133. If I
>
> understand correctly, in addition to supporting the tasks / operators to
>
> exit correctly, now we also want to unify the process of the tasks and
>
> operators for savepoint / finish.
>
> I think the second thing is orthogonal to the FLINK-21133 because there are
>
> topologies that have both the bounded and unbounded input.
>
>
>
> *3. How to unify the operator exit process of FLIP-147 with
>
> stop-with-savepoint?*
>
> I am not very sure about how to do it yet. But if I understand the
>
> discussion in the jira correctly it needs to introduce some logic into
>
> `CheckpointCoordinator`, which responses for triggering “the unified
>
> operator exit process”. Am I correct?
>
>
>
> Best,
>
> Guowei
>
>
>
>
>
> On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann  wrote:
>
>
>
> > Thanks for the explanation Yun and Guowei. I have to admit that I do not
>
> > fully understand why this is strictly required but I think that we are
>
> > touching two very important aspects which might have far fetching
>
> > consequences for how Flink works:
>
> >
>
> > 1) Do we want to allow that multiple checkpoints are required to
>
> > materialize results?
>
> > 2) Do we want to allow to emit records in notifyCheckpointComplete?
>
> >
>
> > For 1) I am not sure whether this has been discussed within the community
>
> > sufficiently. Requiring multiple checkpoints to materialize a result
>
> > because of multi level committers has the consequence that we increase
> the
>
> > latency from checkpoint interval to #levels * checkpoint interval.
>
> > Moreover, having to drain the pipeline in multiple steps, would break the
>
> > stop-with-savepoint --drain because which savepoint do you report to the
>
> > user?
>
> >
>
> > For 2) allowing to send records after the final notifyCheckpointComplete
>
> > will effectively mean that we need to shut down a topology in multiple
>
> > steps (in the worst case one operator per checkpoint). This would be a
>
> > strong argument for not allowing this to me. The fact that users can send
>
> > records after the notifyCheckpointComplete is more by accident than by
>
> > design. I think we should make this a very deliberate decision and in
> doubt
>
> > I would be in favour of a more restrictive model unless there is a very
>
> > good reason why this should be supported.
>
> >
>
> > Taking also the discussion in FLINK-21133 [1] into account, it seems to
> me
>
> > that we haven't really understood what kind of guarantees we want to give
>
> > to our users and how the final checkpoint should exactly work. I
> understand
>
> > that this is not included in the first scope of FLIP-147 but I think this
>
> > is so important that we should figure this out asap. Also because the
> exact
>
> > shut down behaviour will have to be aligned with the lifecycle of a
>
> > Task/StreamTask/StreamOperator. And last but not least because other
>
> > features such as the new sink API start building upon a shut down model
>
> > which has not been fully understood/agreed upon.
>
> >
>
> > [1] https://issues.apache.org/jira/browse/FLINK-21133
>
> >
>
> > Cheers,
>
> > Till
>
> >
>
> > On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma  wrote:
>
> >
>
> > > Thanks Yun for the detailed explanation.
>
> > > A simple supplementary explanation about the sink case: Maybe we could
>
> > use
>
> > > `OperatorCoordinator` to avoid sending the element to the downstream
>
> > > operator.
>
> > > But I agree we could not limit the users not to emit records in the
>
> > > `notiyCheckpointComplete`.
>
> > >
>
> > > Best,
>
> > > Guowei
>
> > >
>
> > >
>
> > > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao
>
> > > wrote:
>
> > >
>
> > > > Hi all,
>
> > > >
>
> > > > I'd like to first detail the issue with emitting records in
>
> > > > notifyCheckpointComplete for context. For specific usage,
>
> > > > an example would be for sink, it might want to write some metadata
>
> > after
>
> > > > all the transactions are committed
>
> > > > (like write a marker file _SUCCESS to the output directory). This
> case
>
> > is
>
> > > > currently supported via the two level
>
> > > > committers of the new sink API: when received endOfInput(), the
>
> > Committer
>
> > > > wait for another checkpoint to
>
> > > > commits all the pending transactions and emit the list of files to
> the
>
> > > > GlobalCommitter. The GlobalCommitter
>
> > > > would wait for another checkpoint to also write the metadata with 2pc
>
> > > > (Although sometimes 2pc is not needed
>
> > > > for writing metadata, it should be only an optimization and still
>
> > > requires
>
> > > > the Committer do commit before
>
> > > > notifying the global Committer. Also another note is GlobalCommitter
> is
>
> > > > also added for some other cases
>
> > > > like some sinks want an commiter with dop = 1, like IceBergSink).
>
> > > >
>
> > > > However, a more general issue to me is that currently we do not limit
>
> > > > users to not emit records in
>
> > > > notifyCheckpointComplete in the API level. The sink case could be
>
> > viewed
>
> > > > as a special case, but in addition
>
> > > > to this one, logically users could also implement their own cases
> that
>
> > > > emits records in notifyCheckpointComplete.
>
> > > >
>
> > > > Best,
>
> > > > Yun
>
> > > >
>
> > > > ------------------Original Mail ------------------
>
> > > > Sender:Arvid Heise
>
> > > > Send Date:Fri Feb 12 20:46:04 2021
>
> > > > Recipients:dev
>
> > > > CC:Yun Gao
>
> > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
>
> > Finished
>
> > > > Hi Piotr,
>
> > > >
>
> > > >
>
> > > >
>
> > > > Thank you for raising your concern. Unfortunately, I do not have a
>
> > better
>
> > > >
>
> > > > idea than doing closing of operators intermittently with checkpoints
> (=
>
> > > >
>
> > > > multiple last checkpoints).
>
> > > >
>
> > > >
>
> > > >
>
> > > > However, two ideas on how to improve the overall user experience:
>
> > > >
>
> > > > 1. If an operator is not relying on notifyCheckpointComplete, we can
>
> > > close
>
> > > >
>
> > > > it faster (without waiting for a checkpoint). In general, I'd assume
>
> > that
>
> > > >
>
> > > > almost all non-sinks behave that way.
>
> > > >
>
> > > > 2. We may increase the checkpointing frequency for the last
>
> > checkpoints.
>
> > > We
>
> > > >
>
> > > > need to avoid overloading checkpoint storages and task managers, but
> I
>
> > > >
>
> > > > assume the more operators are closed, the lower the checkpointing
>
> > > interval
>
> > > >
>
> > > > can be.
>
> > > >
>
> > > >
>
> > > >
>
> > > > For 1, I'd propose to add (name TBD):
>
> > > >
>
> > > >
>
> > > >
>
> > > > default boolean StreamOperator#requiresFinalCheckpoint() {
>
> > > >
>
> > > > return true;
>
> > > >
>
> > > > }
>
> > > >
>
> > > >
>
> > > >
>
> > > > This means all operators are conservatively (=slowly) closed. For
> most
>
> > > >
>
> > > > operators, we can then define their behavior by overriding in
>
> > > >
>
> > > > AbstractUdfStreamOperator
>
> > > >
>
> > > >
>
> > > >
>
> > > > @Override
>
> > > >
>
> > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
>
> > > >
>
> > > > return userFunction instanceof CheckpointListener;
>
> > > >
>
> > > > }
>
> > > >
>
> > > >
>
> > > >
>
> > > > This idea can be further refined in also adding
> requiresFinalCheckpoint
>
> > > to
>
> > > >
>
> > > > CheckpointListener to exclude all operators with UDFs that implement
>
> > > >
>
> > > > CheckpointListener but do not need it for 2pc.
>
> > > >
>
> > > >
>
> > > >
>
> > > > @Override
>
> > > >
>
> > > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
>
> > > >
>
> > > > return userFunction instanceof CheckpointListener &&
>
> > > >
>
> > > > ((CheckpointListener) userFunction).requiresFinalCheckpoint();
>
> > > >
>
> > > > }
>
> > > >
>
> > > >
>
> > > >
>
> > > > That approach would also work for statebackends/snapshot strategies
>
> > that
>
> > > >
>
> > > > require some 2pc.
>
> > > >
>
> > > >
>
> > > >
>
> > > > If we can contain it to the @PublicEvolving StreamOperator, it would
> be
>
> > > >
>
> > > > better of course.
>
> > > >
>
> > > >
>
> > > >
>
> > > > Best,
>
> > > >
>
> > > >
>
> > > >
>
> > > > Arvid
>
> > > >
>
> > > >
>
> > > >
>
> > > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
>
> > > >
>
> > > > wrote:
>
> > > >
>
> > > >
>
> > > >
>
> > > > > Hey,
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > I would like to raise a concern about implementation of the final
>
> > > >
>
> > > > > checkpoints taking into account operators/functions that are
>
> > > implementing
>
> > > >
>
> > > > > two phase commit (2pc) protocol for exactly-once processing with
> some
>
> > > >
>
> > > > > external state (kept outside of the Flink). Primarily exactly-once
>
> > > sinks.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > First of all, as I understand it, this is not planned in the first
>
> > > > version
>
> > > >
>
> > > > > of this FLIP. I'm fine with that, however I would strongly
> emphasize
>
> > > this
>
> > > >
>
> > > > > in every place we will be mentioning FLIP-147 efforts. This is
>
> > because
>
> > > > me,
>
> > > >
>
> > > > > as a user, upon hearing "Flink supports checkpointing with bounded
>
> > > > inputs"
>
> > > >
>
> > > > > I would expect 2pc to work properly and to commit the external side
>
> > > > effects
>
> > > >
>
> > > > > upon finishing. As it is now, I (as a user) would be surprised
> with a
>
> > > >
>
> > > > > silent data loss (of not committed trailing data). This is just a
>
> > > remark,
>
> > > >
>
> > > > > that we need to attach this warning to every blog
>
> > > post/documentation/user
>
> > > >
>
> > > > > mailing list response related to "Support Checkpoints After Tasks
>
> > > >
>
> > > > > Finished". Also I would suggest to prioritize the follow up of
>
> > > supporting
>
> > > >
>
> > > > > 2pc.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > Secondly, I think we are missing how difficult and problematic will
>
> > be
>
> > > > 2pc
>
> > > >
>
> > > > > support with the final checkpoint.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > For starters, keep in mind that currently 2pc can be implemented by
>
> > > users
>
> > > >
>
> > > > > using both `@Public` APIs as functions and `@PublicEvolving`
>
> > operators
>
> > > in
>
> > > >
>
> > > > > any place in the job graph. It's not limited to only the sinks. For
>
> > > >
>
> > > > > example users could easily implement the `AsynFunction` (for
>
> > > >
>
> > > > > `AsyncWaitOperator`) that is using 2pc based on the
>
> > > `CheckpointListener`
>
> > > >
>
> > > > > interface. I'm not saying it's common, probably just a tiny
> minority
>
> > of
>
> > > >
>
> > > > > users are doing that (if any at all), but nevertheless that's
>
> > possible
>
> > > > and
>
> > > >
>
> > > > > currently (implicitly?) supported in Flink.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > Next complication is the support of bounded streams
>
> > (`BoundedOneInput`
>
> > > or
>
> > > >
>
> > > > > `BoundedMultiInput` interfaces) and the closing/shutdown procedure
> of
>
> > > the
>
> > > >
>
> > > > > operators. Currently it works as follows:
>
> > > >
>
> > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
>
> > > >
>
> > > > > 1. `endOfInput` is called on the first operator in the chain
>
> > > >
>
> > > > > 2. We quiesce the processing timers
>
> > > >
>
> > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for
> the
>
> > > > first
>
> > > >
>
> > > > > operator, so no new timers will be triggered
>
> > > >
>
> > > > > 3. We wait for the already fired timers to finish executing
> (spinning
>
> > > >
>
> > > > > mailbox loop)
>
> > > >
>
> > > > > 4. We are closing the first operator
>
> > > >
>
> > > > > 5. We go to the next (second) operator in the chain and repeat the
>
> > > steps
>
> > > > 1.
>
> > > >
>
> > > > > to 5.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > This is because operators can emit data after processing
>
> > `endOfInput`,
>
> > > > from
>
> > > >
>
> > > > > timers, async mailbox actions and inside the `close` method itself.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > Now the problem is to support the final checkpoint with 2pc, we
> need
>
> > > >
>
> > > > > trigger `snapshotState` and `notifyCheckpointComplete` call at the
>
> > very
>
> > > >
>
> > > > > least only after `endOfInput` call on the operator. Probably the
> best
>
> > > > place
>
> > > >
>
> > > > > would be in between steps 3. and 4. However that means, we would be
>
> > > > forced
>
> > > >
>
> > > > > to wait for steps 1. to 3. to finish, then wait for a next
> checkpoint
>
> > > to
>
> > > >
>
> > > > > trigger AND complete, before finally closing the head operator, and
>
> > > only
>
> > > >
>
> > > > > then we can start closing the next operator in the chain:
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > 0. Task receives EndOfPartitionEvent (or source finishes)
>
> > > >
>
> > > > > 1. `endOfInput` is called on the first operator in the chain
>
> > > >
>
> > > > > 2. We quiesce the processing timers
>
> > > >
>
> > > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for
> the
>
> > > > first
>
> > > >
>
> > > > > operator, so no new timers will be triggered
>
> > > >
>
> > > > > 3. We wait for the already fired timers to finish executing
> (spinning
>
> > > >
>
> > > > > mailbox loop)
>
> > > >
>
> > > > > *3b. We wait for one more checkpoint to trigger and for the
>
> > > >
>
> > > > > `notifyCheckpointComplete` RPC.*
>
> > > >
>
> > > > > 4. We are closing the first operator
>
> > > >
>
> > > > > 5. We go to the next (second) operator in the chain and repeat the
>
> > > steps
>
> > > > 1.
>
> > > >
>
> > > > > to 5.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > That means, we can close one operator per successful checkpoint. To
>
> > > close
>
> > > >
>
> > > > > 10 operators, we would need 10 successful checkpoints.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > I was thinking about different approaches to this problem, and I
>
> > > couldn't
>
> > > >
>
> > > > > find any viable ones. All I could think of would break the current
>
> > > >
>
> > > > > `@Public` API and/or would be ugly/confusing for the users.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > For example a relatively simple solution, to introduce a `preClose`
>
> > or
>
> > > >
>
> > > > > `flush` method to the operators, with a contract that after
>
> > > >
>
> > > > > `flush`, operators would be forbidden from emitting more records,
> so
>
> > > that
>
> > > >
>
> > > > > we can replace step 4. with this `flush` call, and then having a
>
> > single
>
> > > >
>
> > > > > checkpoint to finish 2pc for all of the operators inside the chain,
>
> > > > doesn't
>
> > > >
>
> > > > > work. Sheer fact of adding this `flush` method and changing the
>
> > > contract
>
> > > >
>
> > > > > would break the current API and Yun Gao has pointed out to me, that
>
> > we
>
> > > >
>
> > > > > either already support, or want to support operators that are
>
> > emitting
>
> > > >
>
> > > > > records from within the `notifyCheckpointComplete` call:
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > > Yun Gao:
>
> > > >
>
> > > > > > like with the new sink api there might be writer -> committer ->
>
> > > global
>
> > > >
>
> > > > > committer, the committer would need to wait for the last checkpoint
>
> > to
>
> > > >
>
> > > > > commit
>
> > > >
>
> > > > > > the last piece of data, and after that it also need to emit the
>
> > list
>
> > > of
>
> > > >
>
> > > > > transactions get committed to global committer to do some
>
> > finalization
>
> > > >
>
> > > > > logic.
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > So it wouldn't solve the problem (at least not fully).
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > I don't know if anyone has any better ideas how to solve this
>
> > problem?
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > Piotrek
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > pt., 15 sty 2021 o 14:57 Yun Gao
>
> > > >
>
> > > > > napisał(a):
>
> > > >
>
> > > > >
>
> > > >
>
> > > > > > Hi Aljoscha,
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > > I think so since we seems to do not have other divergence and new
>
> > > >
>
> > > > > > objections now. I'll open the vote then. Very thanks!
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > > Best,
>
> > > >
>
> > > > > > Yun
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > >
> ------------------------------------------------------------------
>
> > > >
>
> > > > > > From:Aljoscha Krettek
>
> > > >
>
> > > > > > Send Time:2021 Jan. 15 (Fri.) 21:24
>
> > > >
>
> > > > > > To:dev
>
> > > >
>
> > > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
>
> > > > Finished
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > > Thanks for the summary! I think we can now move towards a [VOTE]
>
> > > > thread,
>
> > > >
>
> > > > > > right?
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > > On 2021/01/15 13:43, Yun Gao wrote:
>
> > > >
>
> > > > > > >1) For the problem that the "new" root task coincidently
> finished
>
> > > >
>
> > > > > > >before getting triggered successfully, we have listed two
> options
>
> > in
>
> > > >
>
> > > > > > >the FLIP-147[1], for the first version, now we are not tend to
> go
>
> > > with
>
> > > >
>
> > > > > > >the first option that JM would re-compute and re-trigger new
>
> > sources
>
> > > >
>
> > > > > > >when it realized some tasks are not triggered successfully. This
>
> > > > option
>
> > > >
>
> > > > > > >would avoid the complexity of adding new PRC and duplicating
> task
>
> > > >
>
> > > > > > >states, and in average case it would not cause too much
> overhead.
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > > You wrote "we are *not* tend to go with the first option", but I
>
> > > think
>
> > > >
>
> > > > > > you meant wo write "we tend to *now* go with the first option",
>
> > > right?
>
> > > >
>
> > > > > > That's also how it is in the FLIP, I just wanted to clarify for
> the
>
> > > >
>
> > > > > > mailing list.
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > > >
>
> > > >
>
> > > > >
>
> > > >
>
> > > >
>
> > >
>
> >
>
>

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi Till, Guowei,

Very thanks for initiating the disucssion and the deep thoughts! 

For the notifyCheckpointComplete, I also agree we could try to avoid
emitting new records in notifyCheckpointComplete via using OperatorCoordinator
for new sink API. Besides, the hive sink might also need some modification
for it also emits records in notifyCheckpointComplete.

For unifying the process of stopping with savepoint and finished due to all records
are processed, I also agree with that unifying would always be better if we could achieve, 
but I'm still not fully catch up with the implementation: Based on the discussion in FLINK-21133, 
my understanding is that for stopping with savepoint, now we want to first stop the source, then we 
trigger a savepoint, and after the source received notifyCheckpointComplete, the source would 
start emitting EndOfPartitionEvent to finish the job, am I correct ?

For normal finish, a difference to me might be if we have multiple sources, we could not guarantee
when the sources are to finish. We might have one source run one 1 minute and another one run for
1 hour. To unify with the process with stop with savepoint, we might need to hold the fast source until
all the sources are finished? An coordinator would be introduced to count the number of sources
runing and trigger the final savepoint / checkpoint. For the extreme cases, if we have both bounded and 
unbounded sources, we might only count how much bounded source are remaining ? And if all the bounded
sources are finished we would trigger the special checkpoint. After all the bounded part of the graph are 
finished, the the remaining part could still do checkpoint and commit data with FLIP-147.

Best,
Yun




 ------------------Original Mail ------------------
Sender:Guowei Ma <gu...@gmail.com>
Send Date:Wed Feb 24 17:35:36 2021
Recipients:dev <de...@flink.apache.org>
CC:Arvid Heise <ar...@apache.org>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
Hi, Till

Thank you very much for your careful consideration



*1. Emit records in `NotifyCheckpointComplete`.*

 Sorry for making you misunderstanding because of my expression. I just

want to say the current interface does not prevent users from doing it.

From the perspective of the new sink api, we might not depend on emitting

records in `NotifyCheckpointComplete`, like using `OperatorCoordinator`

instead.





*2. What does the FLIP-147 guarantee?*I think initially this FLIP want to

achieve two targets:

1. Tasks/Operators exit correctly (as you mentioned the lifecycle of a

Task/StreamTask/StreamOperator.).

2. Continue to trigger checkpoint after some tasks for mixed jobs.



I think the first thing is related to the discussion in FLINK-21133. If I

understand correctly, in addition to supporting the tasks / operators to

exit correctly, now we also want to unify the process of the tasks and

operators for savepoint / finish.

I think the second thing is orthogonal to the FLINK-21133 because there are

topologies that have both the bounded and unbounded input.



*3. How to unify the operator exit process of FLIP-147 with

stop-with-savepoint?*

I am not very sure about how to do it yet. But if I understand the

discussion in the jira correctly it needs to introduce some logic into

`CheckpointCoordinator`, which responses for triggering “the unified

operator exit process”. Am I correct?



Best,

Guowei





On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann  wrote:



> Thanks for the explanation Yun and Guowei. I have to admit that I do not

> fully understand why this is strictly required but I think that we are

> touching two very important aspects which might have far fetching

> consequences for how Flink works:

>

> 1) Do we want to allow that multiple checkpoints are required to

> materialize results?

> 2) Do we want to allow to emit records in notifyCheckpointComplete?

>

> For 1) I am not sure whether this has been discussed within the community

> sufficiently. Requiring multiple checkpoints to materialize a result

> because of multi level committers has the consequence that we increase the

> latency from checkpoint interval to #levels * checkpoint interval.

> Moreover, having to drain the pipeline in multiple steps, would break the

> stop-with-savepoint --drain because which savepoint do you report to the

> user?

>

> For 2) allowing to send records after the final notifyCheckpointComplete

> will effectively mean that we need to shut down a topology in multiple

> steps (in the worst case one operator per checkpoint). This would be a

> strong argument for not allowing this to me. The fact that users can send

> records after the notifyCheckpointComplete is more by accident than by

> design. I think we should make this a very deliberate decision and in doubt

> I would be in favour of a more restrictive model unless there is a very

> good reason why this should be supported.

>

> Taking also the discussion in FLINK-21133 [1] into account, it seems to me

> that we haven't really understood what kind of guarantees we want to give

> to our users and how the final checkpoint should exactly work. I understand

> that this is not included in the first scope of FLIP-147 but I think this

> is so important that we should figure this out asap. Also because the exact

> shut down behaviour will have to be aligned with the lifecycle of a

> Task/StreamTask/StreamOperator. And last but not least because other

> features such as the new sink API start building upon a shut down model

> which has not been fully understood/agreed upon.

>

> [1] https://issues.apache.org/jira/browse/FLINK-21133

>

> Cheers,

> Till

>

> On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma  wrote:

>

> > Thanks Yun for the detailed explanation.

> > A simple supplementary explanation about the sink case: Maybe we could

> use

> > `OperatorCoordinator` to avoid sending the element to the downstream

> > operator.

> > But I agree we could not limit the users not to emit records in the

> > `notiyCheckpointComplete`.

> >

> > Best,

> > Guowei

> >

> >

> > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao 

> > wrote:

> >

> > > Hi all,

> > >

> > > I'd like to first detail the issue with emitting records in

> > > notifyCheckpointComplete for context. For specific usage,

> > > an example would be for sink, it might want to write some metadata

> after

> > > all the transactions are committed

> > > (like write a marker file _SUCCESS to the output directory). This case

> is

> > > currently supported via the two level

> > > committers of the new sink API: when received endOfInput(), the

> Committer

> > > wait for another checkpoint to

> > > commits all the pending transactions and emit the list of files to the

> > > GlobalCommitter. The GlobalCommitter

> > > would wait for another checkpoint to also write the metadata with 2pc

> > > (Although sometimes 2pc is not needed

> > > for writing metadata, it should be only an optimization and still

> > requires

> > > the Committer do commit before

> > > notifying the global Committer. Also another note is GlobalCommitter is

> > > also added for some other cases

> > > like some sinks want an commiter with dop = 1, like IceBergSink).

> > >

> > > However, a more general issue to me is that currently we do not limit

> > > users to not emit records in

> > > notifyCheckpointComplete in the API level. The sink case could be

> viewed

> > > as a special case, but in addition

> > > to this one, logically users could also implement their own cases that

> > > emits records in notifyCheckpointComplete.

> > >

> > > Best,

> > > Yun

> > >

> > > ------------------Original Mail ------------------

> > > Sender:Arvid Heise 

> > > Send Date:Fri Feb 12 20:46:04 2021

> > > Recipients:dev 

> > > CC:Yun Gao 

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

> Finished

> > > Hi Piotr,

> > >

> > >

> > >

> > > Thank you for raising your concern. Unfortunately, I do not have a

> better

> > >

> > > idea than doing closing of operators intermittently with checkpoints (=

> > >

> > > multiple last checkpoints).

> > >

> > >

> > >

> > > However, two ideas on how to improve the overall user experience:

> > >

> > > 1. If an operator is not relying on notifyCheckpointComplete, we can

> > close

> > >

> > > it faster (without waiting for a checkpoint). In general, I'd assume

> that

> > >

> > > almost all non-sinks behave that way.

> > >

> > > 2. We may increase the checkpointing frequency for the last

> checkpoints.

> > We

> > >

> > > need to avoid overloading checkpoint storages and task managers, but I

> > >

> > > assume the more operators are closed, the lower the checkpointing

> > interval

> > >

> > > can be.

> > >

> > >

> > >

> > > For 1, I'd propose to add (name TBD):

> > >

> > >

> > >

> > > default boolean StreamOperator#requiresFinalCheckpoint() {

> > >

> > > return true;

> > >

> > > }

> > >

> > >

> > >

> > > This means all operators are conservatively (=slowly) closed. For most

> > >

> > > operators, we can then define their behavior by overriding in

> > >

> > > AbstractUdfStreamOperator

> > >

> > >

> > >

> > > @Override

> > >

> > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {

> > >

> > > return userFunction instanceof CheckpointListener;

> > >

> > > }

> > >

> > >

> > >

> > > This idea can be further refined in also adding requiresFinalCheckpoint

> > to

> > >

> > > CheckpointListener to exclude all operators with UDFs that implement

> > >

> > > CheckpointListener but do not need it for 2pc.

> > >

> > >

> > >

> > > @Override

> > >

> > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {

> > >

> > > return userFunction instanceof CheckpointListener &&

> > >

> > > ((CheckpointListener) userFunction).requiresFinalCheckpoint();

> > >

> > > }

> > >

> > >

> > >

> > > That approach would also work for statebackends/snapshot strategies

> that

> > >

> > > require some 2pc.

> > >

> > >

> > >

> > > If we can contain it to the @PublicEvolving StreamOperator, it would be

> > >

> > > better of course.

> > >

> > >

> > >

> > > Best,

> > >

> > >

> > >

> > > Arvid

> > >

> > >

> > >

> > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski

> > >

> > > wrote:

> > >

> > >

> > >

> > > > Hey,

> > >

> > > >

> > >

> > > > I would like to raise a concern about implementation of the final

> > >

> > > > checkpoints taking into account operators/functions that are

> > implementing

> > >

> > > > two phase commit (2pc) protocol for exactly-once processing with some

> > >

> > > > external state (kept outside of the Flink). Primarily exactly-once

> > sinks.

> > >

> > > >

> > >

> > > > First of all, as I understand it, this is not planned in the first

> > > version

> > >

> > > > of this FLIP. I'm fine with that, however I would strongly emphasize

> > this

> > >

> > > > in every place we will be mentioning FLIP-147 efforts. This is

> because

> > > me,

> > >

> > > > as a user, upon hearing "Flink supports checkpointing with bounded

> > > inputs"

> > >

> > > > I would expect 2pc to work properly and to commit the external side

> > > effects

> > >

> > > > upon finishing. As it is now, I (as a user) would be surprised with a

> > >

> > > > silent data loss (of not committed trailing data). This is just a

> > remark,

> > >

> > > > that we need to attach this warning to every blog

> > post/documentation/user

> > >

> > > > mailing list response related to "Support Checkpoints After Tasks

> > >

> > > > Finished". Also I would suggest to prioritize the follow up of

> > supporting

> > >

> > > > 2pc.

> > >

> > > >

> > >

> > > > Secondly, I think we are missing how difficult and problematic will

> be

> > > 2pc

> > >

> > > > support with the final checkpoint.

> > >

> > > >

> > >

> > > > For starters, keep in mind that currently 2pc can be implemented by

> > users

> > >

> > > > using both `@Public` APIs as functions and `@PublicEvolving`

> operators

> > in

> > >

> > > > any place in the job graph. It's not limited to only the sinks. For

> > >

> > > > example users could easily implement the `AsynFunction` (for

> > >

> > > > `AsyncWaitOperator`) that is using 2pc based on the

> > `CheckpointListener`

> > >

> > > > interface. I'm not saying it's common, probably just a tiny minority

> of

> > >

> > > > users are doing that (if any at all), but nevertheless that's

> possible

> > > and

> > >

> > > > currently (implicitly?) supported in Flink.

> > >

> > > >

> > >

> > > > Next complication is the support of bounded streams

> (`BoundedOneInput`

> > or

> > >

> > > > `BoundedMultiInput` interfaces) and the closing/shutdown procedure of

> > the

> > >

> > > > operators. Currently it works as follows:

> > >

> > > > 0. Task receives EndOfPartitionEvent (or source finishes)

> > >

> > > > 1. `endOfInput` is called on the first operator in the chain

> > >

> > > > 2. We quiesce the processing timers

> > >

> > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for the

> > > first

> > >

> > > > operator, so no new timers will be triggered

> > >

> > > > 3. We wait for the already fired timers to finish executing (spinning

> > >

> > > > mailbox loop)

> > >

> > > > 4. We are closing the first operator

> > >

> > > > 5. We go to the next (second) operator in the chain and repeat the

> > steps

> > > 1.

> > >

> > > > to 5.

> > >

> > > >

> > >

> > > > This is because operators can emit data after processing

> `endOfInput`,

> > > from

> > >

> > > > timers, async mailbox actions and inside the `close` method itself.

> > >

> > > >

> > >

> > > > Now the problem is to support the final checkpoint with 2pc, we need

> > >

> > > > trigger `snapshotState` and `notifyCheckpointComplete` call at the

> very

> > >

> > > > least only after `endOfInput` call on the operator. Probably the best

> > > place

> > >

> > > > would be in between steps 3. and 4. However that means, we would be

> > > forced

> > >

> > > > to wait for steps 1. to 3. to finish, then wait for a next checkpoint

> > to

> > >

> > > > trigger AND complete, before finally closing the head operator, and

> > only

> > >

> > > > then we can start closing the next operator in the chain:

> > >

> > > >

> > >

> > > > 0. Task receives EndOfPartitionEvent (or source finishes)

> > >

> > > > 1. `endOfInput` is called on the first operator in the chain

> > >

> > > > 2. We quiesce the processing timers

> > >

> > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for the

> > > first

> > >

> > > > operator, so no new timers will be triggered

> > >

> > > > 3. We wait for the already fired timers to finish executing (spinning

> > >

> > > > mailbox loop)

> > >

> > > > *3b. We wait for one more checkpoint to trigger and for the

> > >

> > > > `notifyCheckpointComplete` RPC.*

> > >

> > > > 4. We are closing the first operator

> > >

> > > > 5. We go to the next (second) operator in the chain and repeat the

> > steps

> > > 1.

> > >

> > > > to 5.

> > >

> > > >

> > >

> > > > That means, we can close one operator per successful checkpoint. To

> > close

> > >

> > > > 10 operators, we would need 10 successful checkpoints.

> > >

> > > >

> > >

> > > > I was thinking about different approaches to this problem, and I

> > couldn't

> > >

> > > > find any viable ones. All I could think of would break the current

> > >

> > > > `@Public` API and/or would be ugly/confusing for the users.

> > >

> > > >

> > >

> > > > For example a relatively simple solution, to introduce a `preClose`

> or

> > >

> > > > `flush` method to the operators, with a contract that after

> > >

> > > > `flush`, operators would be forbidden from emitting more records, so

> > that

> > >

> > > > we can replace step 4. with this `flush` call, and then having a

> single

> > >

> > > > checkpoint to finish 2pc for all of the operators inside the chain,

> > > doesn't

> > >

> > > > work. Sheer fact of adding this `flush` method and changing the

> > contract

> > >

> > > > would break the current API and Yun Gao has pointed out to me, that

> we

> > >

> > > > either already support, or want to support operators that are

> emitting

> > >

> > > > records from within the `notifyCheckpointComplete` call:

> > >

> > > >

> > >

> > > > > Yun Gao:

> > >

> > > > > like with the new sink api there might be writer -> committer ->

> > global

> > >

> > > > committer, the committer would need to wait for the last checkpoint

> to

> > >

> > > > commit

> > >

> > > > > the last piece of data, and after that it also need to emit the

> list

> > of

> > >

> > > > transactions get committed to global committer to do some

> finalization

> > >

> > > > logic.

> > >

> > > >

> > >

> > > > So it wouldn't solve the problem (at least not fully).

> > >

> > > >

> > >

> > > > I don't know if anyone has any better ideas how to solve this

> problem?

> > >

> > > >

> > >

> > > > Piotrek

> > >

> > > >

> > >

> > > > pt., 15 sty 2021 o 14:57 Yun Gao

> > >

> > > > napisał(a):

> > >

> > > >

> > >

> > > > > Hi Aljoscha,

> > >

> > > > >

> > >

> > > > > I think so since we seems to do not have other divergence and new

> > >

> > > > > objections now. I'll open the vote then. Very thanks!

> > >

> > > > >

> > >

> > > > > Best,

> > >

> > > > > Yun

> > >

> > > > >

> > >

> > > > >

> > >

> > > > > ------------------------------------------------------------------

> > >

> > > > > From:Aljoscha Krettek

> > >

> > > > > Send Time:2021 Jan. 15 (Fri.) 21:24

> > >

> > > > > To:dev

> > >

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

> > > Finished

> > >

> > > > >

> > >

> > > > > Thanks for the summary! I think we can now move towards a [VOTE]

> > > thread,

> > >

> > > > > right?

> > >

> > > > >

> > >

> > > > > On 2021/01/15 13:43, Yun Gao wrote:

> > >

> > > > > >1) For the problem that the "new" root task coincidently finished

> > >

> > > > > >before getting triggered successfully, we have listed two options

> in

> > >

> > > > > >the FLIP-147[1], for the first version, now we are not tend to go

> > with

> > >

> > > > > >the first option that JM would re-compute and re-trigger new

> sources

> > >

> > > > > >when it realized some tasks are not triggered successfully. This

> > > option

> > >

> > > > > >would avoid the complexity of adding new PRC and duplicating task

> > >

> > > > > >states, and in average case it would not cause too much overhead.

> > >

> > > > >

> > >

> > > > > You wrote "we are *not* tend to go with the first option", but I

> > think

> > >

> > > > > you meant wo write "we tend to *now* go with the first option",

> > right?

> > >

> > > > > That's also how it is in the FLIP, I just wanted to clarify for the

> > >

> > > > > mailing list.

> > >

> > > > >

> > >

> > > > >

> > >

> > > >

> > >

> > >

> >

>


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

Posted by Guowei Ma <gu...@gmail.com>.
Hi, Till
Thank you very much for your careful consideration

*1. Emit records in `NotifyCheckpointComplete`.*
    Sorry for making you misunderstanding because of my expression. I just
want to say the current interface does not prevent users from doing it.
From the perspective of the new sink api, we might not depend on emitting
records in `NotifyCheckpointComplete`, like using `OperatorCoordinator`
instead.


*2. What does the FLIP-147 guarantee?*I think initially this FLIP want to
achieve two targets:
1. Tasks/Operators exit correctly (as you mentioned the lifecycle of a
Task/StreamTask/StreamOperator.).
2. Continue to trigger checkpoint after some tasks for mixed jobs.

I think the first thing is related to the discussion in FLINK-21133. If I
understand correctly, in addition to supporting the tasks / operators to
exit correctly, now we also want to unify the process of the tasks and
operators for savepoint / finish.
I think the second thing is orthogonal to the FLINK-21133 because there are
topologies that have both the bounded and unbounded input.

*3. How to unify the operator exit process of FLIP-147 with
stop-with-savepoint?*
I  am not very sure about how to do it yet. But if I understand the
discussion in the jira correctly it needs to introduce some logic into
`CheckpointCoordinator`, which responses for triggering “the unified
operator exit process”.  Am I correct?

Best,
Guowei


On Tue, Feb 23, 2021 at 5:10 PM Till Rohrmann <tr...@apache.org> wrote:

> Thanks for the explanation Yun and Guowei. I have to admit that I do not
> fully understand why this is strictly required but I think that we are
> touching two very important aspects which might have far fetching
> consequences for how Flink works:
>
> 1) Do we want to allow that multiple checkpoints are required to
> materialize results?
> 2) Do we want to allow to emit records in notifyCheckpointComplete?
>
> For 1) I am not sure whether this has been discussed within the community
> sufficiently. Requiring multiple checkpoints to materialize a result
> because of multi level committers has the consequence that we increase the
> latency from checkpoint interval to #levels * checkpoint interval.
> Moreover, having to drain the pipeline in multiple steps, would break the
> stop-with-savepoint --drain because which savepoint do you report to the
> user?
>
> For 2) allowing to send records after the final notifyCheckpointComplete
> will effectively mean that we need to shut down a topology in multiple
> steps (in the worst case one operator per checkpoint). This would be a
> strong argument for not allowing this to me. The fact that users can send
> records after the notifyCheckpointComplete is more by accident than by
> design. I think we should make this a very deliberate decision and in doubt
> I would be in favour of a more restrictive model unless there is a very
> good reason why this should be supported.
>
> Taking also the discussion in FLINK-21133 [1] into account, it seems to me
> that we haven't really understood what kind of guarantees we want to give
> to our users and how the final checkpoint should exactly work. I understand
> that this is not included in the first scope of FLIP-147 but I think this
> is so important that we should figure this out asap. Also because the exact
> shut down behaviour will have to be aligned with the lifecycle of a
> Task/StreamTask/StreamOperator. And last but not least because other
> features such as the new sink API start building upon a shut down model
> which has not been fully understood/agreed upon.
>
> [1] https://issues.apache.org/jira/browse/FLINK-21133
>
> Cheers,
> Till
>
> On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma <gu...@gmail.com> wrote:
>
> > Thanks Yun for the detailed explanation.
> > A simple supplementary explanation about the sink case: Maybe we could
> use
> > `OperatorCoordinator` to avoid sending the element to the downstream
> > operator.
> > But I agree we could not limit the users not to emit records in the
> > `notiyCheckpointComplete`.
> >
> > Best,
> > Guowei
> >
> >
> > On Tue, Feb 16, 2021 at 2:06 PM Yun Gao <yu...@aliyun.com.invalid>
> > wrote:
> >
> > > Hi all,
> > >
> > > I'd like to first detail the issue with emitting records in
> > > notifyCheckpointComplete for context. For specific usage,
> > > an example would be for sink, it might want to write some metadata
> after
> > > all the transactions are committed
> > > (like write a marker file _SUCCESS to the output directory). This case
> is
> > > currently supported via the two level
> > > committers of the new sink API: when received endOfInput(), the
> Committer
> > > wait for another checkpoint to
> > > commits all the pending transactions and emit the list of files to the
> > > GlobalCommitter. The GlobalCommitter
> > > would wait for another checkpoint to also write the metadata with 2pc
> > > (Although sometimes 2pc is not needed
> > > for writing metadata, it should be only an optimization and still
> > requires
> > > the Committer do commit before
> > > notifying the global Committer. Also another note is GlobalCommitter is
> > > also added for some other cases
> > > like some sinks want an commiter with dop = 1, like IceBergSink).
> > >
> > > However, a more general issue to me is that currently we do not limit
> > > users to not emit records in
> > > notifyCheckpointComplete in the API level. The sink case could be
> viewed
> > > as a special case, but in addition
> > > to this one, logically users could also implement their own cases that
> > > emits records in notifyCheckpointComplete.
> > >
> > > Best,
> > > Yun
> > >
> > >  ------------------Original Mail ------------------
> > > Sender:Arvid Heise <ar...@apache.org>
> > > Send Date:Fri Feb 12 20:46:04 2021
> > > Recipients:dev <de...@flink.apache.org>
> > > CC:Yun Gao <yu...@aliyun.com>
> > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> Finished
> > > Hi Piotr,
> > >
> > >
> > >
> > > Thank you for raising your concern. Unfortunately, I do not have a
> better
> > >
> > > idea than doing closing of operators intermittently with checkpoints (=
> > >
> > > multiple last checkpoints).
> > >
> > >
> > >
> > > However, two ideas on how to improve the overall user experience:
> > >
> > > 1. If an operator is not relying on notifyCheckpointComplete, we can
> > close
> > >
> > > it faster (without waiting for a checkpoint). In general, I'd assume
> that
> > >
> > > almost all non-sinks behave that way.
> > >
> > > 2. We may increase the checkpointing frequency for the last
> checkpoints.
> > We
> > >
> > > need to avoid overloading checkpoint storages and task managers, but I
> > >
> > > assume the more operators are closed, the lower the checkpointing
> > interval
> > >
> > > can be.
> > >
> > >
> > >
> > > For 1, I'd propose to add (name TBD):
> > >
> > >
> > >
> > > default boolean StreamOperator#requiresFinalCheckpoint() {
> > >
> > >  return true;
> > >
> > > }
> > >
> > >
> > >
> > > This means all operators are conservatively (=slowly) closed. For most
> > >
> > > operators, we can then define their behavior by overriding in
> > >
> > > AbstractUdfStreamOperator
> > >
> > >
> > >
> > > @Override
> > >
> > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> > >
> > >  return userFunction instanceof CheckpointListener;
> > >
> > > }
> > >
> > >
> > >
> > > This idea can be further refined in also adding requiresFinalCheckpoint
> > to
> > >
> > > CheckpointListener to exclude all operators with UDFs that implement
> > >
> > > CheckpointListener but do not need it for 2pc.
> > >
> > >
> > >
> > > @Override
> > >
> > > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> > >
> > >  return userFunction instanceof CheckpointListener &&
> > >
> > >  ((CheckpointListener) userFunction).requiresFinalCheckpoint();
> > >
> > > }
> > >
> > >
> > >
> > > That approach would also work for statebackends/snapshot strategies
> that
> > >
> > > require some 2pc.
> > >
> > >
> > >
> > > If we can contain it to the @PublicEvolving StreamOperator, it would be
> > >
> > > better of course.
> > >
> > >
> > >
> > > Best,
> > >
> > >
> > >
> > > Arvid
> > >
> > >
> > >
> > > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
> > >
> > > wrote:
> > >
> > >
> > >
> > > > Hey,
> > >
> > > >
> > >
> > > > I would like to raise a concern about implementation of the final
> > >
> > > > checkpoints taking into account operators/functions that are
> > implementing
> > >
> > > > two phase commit (2pc) protocol for exactly-once processing with some
> > >
> > > > external state (kept outside of the Flink). Primarily exactly-once
> > sinks.
> > >
> > > >
> > >
> > > > First of all, as I understand it, this is not planned in the first
> > > version
> > >
> > > > of this FLIP. I'm fine with that, however I would strongly emphasize
> > this
> > >
> > > > in every place we will be mentioning FLIP-147 efforts. This is
> because
> > > me,
> > >
> > > > as a user, upon hearing "Flink supports checkpointing with bounded
> > > inputs"
> > >
> > > > I would expect 2pc to work properly and to commit the external side
> > > effects
> > >
> > > > upon finishing. As it is now, I (as a user) would be surprised with a
> > >
> > > > silent data loss (of not committed trailing data). This is just a
> > remark,
> > >
> > > > that we need to attach this warning to every blog
> > post/documentation/user
> > >
> > > > mailing list response related to "Support Checkpoints After Tasks
> > >
> > > > Finished". Also I would suggest to prioritize the follow up of
> > supporting
> > >
> > > > 2pc.
> > >
> > > >
> > >
> > > > Secondly, I think we are missing how difficult and problematic will
> be
> > > 2pc
> > >
> > > > support with the final checkpoint.
> > >
> > > >
> > >
> > > > For starters, keep in mind that currently 2pc can be implemented by
> > users
> > >
> > > > using both `@Public` APIs as functions and `@PublicEvolving`
> operators
> > in
> > >
> > > > any place in the job graph. It's not limited to only the sinks. For
> > >
> > > > example users could easily implement the `AsynFunction` (for
> > >
> > > > `AsyncWaitOperator`) that is using 2pc based on the
> > `CheckpointListener`
> > >
> > > > interface. I'm not saying it's common, probably just a tiny minority
> of
> > >
> > > > users are doing that (if any at all), but nevertheless that's
> possible
> > > and
> > >
> > > > currently (implicitly?) supported in Flink.
> > >
> > > >
> > >
> > > > Next complication is the support of bounded streams
> (`BoundedOneInput`
> > or
> > >
> > > > `BoundedMultiInput` interfaces) and the closing/shutdown procedure of
> > the
> > >
> > > > operators. Currently it works as follows:
> > >
> > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> > >
> > > > 1. `endOfInput` is called on the first operator in the chain
> > >
> > > > 2. We quiesce the processing timers
> > >
> > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for the
> > > first
> > >
> > > > operator, so no new timers will be triggered
> > >
> > > > 3. We wait for the already fired timers to finish executing (spinning
> > >
> > > > mailbox loop)
> > >
> > > > 4. We are closing the first operator
> > >
> > > > 5. We go to the next (second) operator in the chain and repeat the
> > steps
> > > 1.
> > >
> > > > to 5.
> > >
> > > >
> > >
> > > > This is because operators can emit data after processing
> `endOfInput`,
> > > from
> > >
> > > > timers, async mailbox actions and inside the `close` method itself.
> > >
> > > >
> > >
> > > > Now the problem is to support the final checkpoint with 2pc, we need
> > >
> > > > trigger `snapshotState` and `notifyCheckpointComplete` call at the
> very
> > >
> > > > least only after `endOfInput` call on the operator. Probably the best
> > > place
> > >
> > > > would be in between steps 3. and 4. However that means, we would be
> > > forced
> > >
> > > > to wait for steps 1. to 3. to finish, then wait for a next checkpoint
> > to
> > >
> > > > trigger AND complete, before finally closing the head operator, and
> > only
> > >
> > > > then we can start closing the next operator in the chain:
> > >
> > > >
> > >
> > > > 0. Task receives EndOfPartitionEvent (or source finishes)
> > >
> > > > 1. `endOfInput` is called on the first operator in the chain
> > >
> > > > 2. We quiesce the processing timers
> > >
> > > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for the
> > > first
> > >
> > > > operator, so no new timers will be triggered
> > >
> > > > 3. We wait for the already fired timers to finish executing (spinning
> > >
> > > > mailbox loop)
> > >
> > > > *3b. We wait for one more checkpoint to trigger and for the
> > >
> > > > `notifyCheckpointComplete` RPC.*
> > >
> > > > 4. We are closing the first operator
> > >
> > > > 5. We go to the next (second) operator in the chain and repeat the
> > steps
> > > 1.
> > >
> > > > to 5.
> > >
> > > >
> > >
> > > > That means, we can close one operator per successful checkpoint. To
> > close
> > >
> > > > 10 operators, we would need 10 successful checkpoints.
> > >
> > > >
> > >
> > > > I was thinking about different approaches to this problem, and I
> > couldn't
> > >
> > > > find any viable ones. All I could think of would break the current
> > >
> > > > `@Public` API and/or would be ugly/confusing for the users.
> > >
> > > >
> > >
> > > > For example a relatively simple solution, to introduce a `preClose`
> or
> > >
> > > > `flush` method to the operators, with a contract that after
> > >
> > > > `flush`, operators would be forbidden from emitting more records, so
> > that
> > >
> > > > we can replace step 4. with this `flush` call, and then having a
> single
> > >
> > > > checkpoint to finish 2pc for all of the operators inside the chain,
> > > doesn't
> > >
> > > > work. Sheer fact of adding this `flush` method and changing the
> > contract
> > >
> > > > would break the current API and Yun Gao has pointed out to me, that
> we
> > >
> > > > either already support, or want to support operators that are
> emitting
> > >
> > > > records from within the `notifyCheckpointComplete` call:
> > >
> > > >
> > >
> > > > > Yun Gao:
> > >
> > > > > like with the new sink api there might be writer -> committer ->
> > global
> > >
> > > > committer, the committer would need to wait for the last checkpoint
> to
> > >
> > > > commit
> > >
> > > > > the last piece of data, and after that it also need to emit the
> list
> > of
> > >
> > > > transactions get committed to global committer to do some
> finalization
> > >
> > > > logic.
> > >
> > > >
> > >
> > > > So it wouldn't solve the problem (at least not fully).
> > >
> > > >
> > >
> > > > I don't know if anyone has any better ideas how to solve this
> problem?
> > >
> > > >
> > >
> > > > Piotrek
> > >
> > > >
> > >
> > > > pt., 15 sty 2021 o 14:57 Yun Gao
> > >
> > > > napisał(a):
> > >
> > > >
> > >
> > > > > Hi Aljoscha,
> > >
> > > > >
> > >
> > > > > I think so since we seems to do not have other divergence and new
> > >
> > > > > objections now. I'll open the vote then. Very thanks!
> > >
> > > > >
> > >
> > > > > Best,
> > >
> > > > > Yun
> > >
> > > > >
> > >
> > > > >
> > >
> > > > > ------------------------------------------------------------------
> > >
> > > > > From:Aljoscha Krettek
> > >
> > > > > Send Time:2021 Jan. 15 (Fri.) 21:24
> > >
> > > > > To:dev
> > >
> > > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > > Finished
> > >
> > > > >
> > >
> > > > > Thanks for the summary! I think we can now move towards a [VOTE]
> > > thread,
> > >
> > > > > right?
> > >
> > > > >
> > >
> > > > > On 2021/01/15 13:43, Yun Gao wrote:
> > >
> > > > > >1) For the problem that the "new" root task coincidently finished
> > >
> > > > > >before getting triggered successfully, we have listed two options
> in
> > >
> > > > > >the FLIP-147[1], for the first version, now we are not tend to go
> > with
> > >
> > > > > >the first option that JM would re-compute and re-trigger new
> sources
> > >
> > > > > >when it realized some tasks are not triggered successfully. This
> > > option
> > >
> > > > > >would avoid the complexity of adding new PRC and duplicating task
> > >
> > > > > >states, and in average case it would not cause too much overhead.
> > >
> > > > >
> > >
> > > > > You wrote "we are *not* tend to go with the first option", but I
> > think
> > >
> > > > > you meant wo write "we tend to *now* go with the first option",
> > right?
> > >
> > > > > That's also how it is in the FLIP, I just wanted to clarify for the
> > >
> > > > > mailing list.
> > >
> > > > >
> > >
> > > > >
> > >
> > > >
> > >
> > >
> >
>

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

Posted by Till Rohrmann <tr...@apache.org>.
Thanks for the explanation Yun and Guowei. I have to admit that I do not
fully understand why this is strictly required but I think that we are
touching two very important aspects which might have far fetching
consequences for how Flink works:

1) Do we want to allow that multiple checkpoints are required to
materialize results?
2) Do we want to allow to emit records in notifyCheckpointComplete?

For 1) I am not sure whether this has been discussed within the community
sufficiently. Requiring multiple checkpoints to materialize a result
because of multi level committers has the consequence that we increase the
latency from checkpoint interval to #levels * checkpoint interval.
Moreover, having to drain the pipeline in multiple steps, would break the
stop-with-savepoint --drain because which savepoint do you report to the
user?

For 2) allowing to send records after the final notifyCheckpointComplete
will effectively mean that we need to shut down a topology in multiple
steps (in the worst case one operator per checkpoint). This would be a
strong argument for not allowing this to me. The fact that users can send
records after the notifyCheckpointComplete is more by accident than by
design. I think we should make this a very deliberate decision and in doubt
I would be in favour of a more restrictive model unless there is a very
good reason why this should be supported.

Taking also the discussion in FLINK-21133 [1] into account, it seems to me
that we haven't really understood what kind of guarantees we want to give
to our users and how the final checkpoint should exactly work. I understand
that this is not included in the first scope of FLIP-147 but I think this
is so important that we should figure this out asap. Also because the exact
shut down behaviour will have to be aligned with the lifecycle of a
Task/StreamTask/StreamOperator. And last but not least because other
features such as the new sink API start building upon a shut down model
which has not been fully understood/agreed upon.

[1] https://issues.apache.org/jira/browse/FLINK-21133

Cheers,
Till

On Tue, Feb 16, 2021 at 9:45 AM Guowei Ma <gu...@gmail.com> wrote:

> Thanks Yun for the detailed explanation.
> A simple supplementary explanation about the sink case: Maybe we could use
> `OperatorCoordinator` to avoid sending the element to the downstream
> operator.
> But I agree we could not limit the users not to emit records in the
> `notiyCheckpointComplete`.
>
> Best,
> Guowei
>
>
> On Tue, Feb 16, 2021 at 2:06 PM Yun Gao <yu...@aliyun.com.invalid>
> wrote:
>
> > Hi all,
> >
> > I'd like to first detail the issue with emitting records in
> > notifyCheckpointComplete for context. For specific usage,
> > an example would be for sink, it might want to write some metadata after
> > all the transactions are committed
> > (like write a marker file _SUCCESS to the output directory). This case is
> > currently supported via the two level
> > committers of the new sink API: when received endOfInput(), the Committer
> > wait for another checkpoint to
> > commits all the pending transactions and emit the list of files to the
> > GlobalCommitter. The GlobalCommitter
> > would wait for another checkpoint to also write the metadata with 2pc
> > (Although sometimes 2pc is not needed
> > for writing metadata, it should be only an optimization and still
> requires
> > the Committer do commit before
> > notifying the global Committer. Also another note is GlobalCommitter is
> > also added for some other cases
> > like some sinks want an commiter with dop = 1, like IceBergSink).
> >
> > However, a more general issue to me is that currently we do not limit
> > users to not emit records in
> > notifyCheckpointComplete in the API level. The sink case could be viewed
> > as a special case, but in addition
> > to this one, logically users could also implement their own cases that
> > emits records in notifyCheckpointComplete.
> >
> > Best,
> > Yun
> >
> >  ------------------Original Mail ------------------
> > Sender:Arvid Heise <ar...@apache.org>
> > Send Date:Fri Feb 12 20:46:04 2021
> > Recipients:dev <de...@flink.apache.org>
> > CC:Yun Gao <yu...@aliyun.com>
> > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
> > Hi Piotr,
> >
> >
> >
> > Thank you for raising your concern. Unfortunately, I do not have a better
> >
> > idea than doing closing of operators intermittently with checkpoints (=
> >
> > multiple last checkpoints).
> >
> >
> >
> > However, two ideas on how to improve the overall user experience:
> >
> > 1. If an operator is not relying on notifyCheckpointComplete, we can
> close
> >
> > it faster (without waiting for a checkpoint). In general, I'd assume that
> >
> > almost all non-sinks behave that way.
> >
> > 2. We may increase the checkpointing frequency for the last checkpoints.
> We
> >
> > need to avoid overloading checkpoint storages and task managers, but I
> >
> > assume the more operators are closed, the lower the checkpointing
> interval
> >
> > can be.
> >
> >
> >
> > For 1, I'd propose to add (name TBD):
> >
> >
> >
> > default boolean StreamOperator#requiresFinalCheckpoint() {
> >
> >  return true;
> >
> > }
> >
> >
> >
> > This means all operators are conservatively (=slowly) closed. For most
> >
> > operators, we can then define their behavior by overriding in
> >
> > AbstractUdfStreamOperator
> >
> >
> >
> > @Override
> >
> > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> >
> >  return userFunction instanceof CheckpointListener;
> >
> > }
> >
> >
> >
> > This idea can be further refined in also adding requiresFinalCheckpoint
> to
> >
> > CheckpointListener to exclude all operators with UDFs that implement
> >
> > CheckpointListener but do not need it for 2pc.
> >
> >
> >
> > @Override
> >
> > boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
> >
> >  return userFunction instanceof CheckpointListener &&
> >
> >  ((CheckpointListener) userFunction).requiresFinalCheckpoint();
> >
> > }
> >
> >
> >
> > That approach would also work for statebackends/snapshot strategies that
> >
> > require some 2pc.
> >
> >
> >
> > If we can contain it to the @PublicEvolving StreamOperator, it would be
> >
> > better of course.
> >
> >
> >
> > Best,
> >
> >
> >
> > Arvid
> >
> >
> >
> > On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
> >
> > wrote:
> >
> >
> >
> > > Hey,
> >
> > >
> >
> > > I would like to raise a concern about implementation of the final
> >
> > > checkpoints taking into account operators/functions that are
> implementing
> >
> > > two phase commit (2pc) protocol for exactly-once processing with some
> >
> > > external state (kept outside of the Flink). Primarily exactly-once
> sinks.
> >
> > >
> >
> > > First of all, as I understand it, this is not planned in the first
> > version
> >
> > > of this FLIP. I'm fine with that, however I would strongly emphasize
> this
> >
> > > in every place we will be mentioning FLIP-147 efforts. This is because
> > me,
> >
> > > as a user, upon hearing "Flink supports checkpointing with bounded
> > inputs"
> >
> > > I would expect 2pc to work properly and to commit the external side
> > effects
> >
> > > upon finishing. As it is now, I (as a user) would be surprised with a
> >
> > > silent data loss (of not committed trailing data). This is just a
> remark,
> >
> > > that we need to attach this warning to every blog
> post/documentation/user
> >
> > > mailing list response related to "Support Checkpoints After Tasks
> >
> > > Finished". Also I would suggest to prioritize the follow up of
> supporting
> >
> > > 2pc.
> >
> > >
> >
> > > Secondly, I think we are missing how difficult and problematic will be
> > 2pc
> >
> > > support with the final checkpoint.
> >
> > >
> >
> > > For starters, keep in mind that currently 2pc can be implemented by
> users
> >
> > > using both `@Public` APIs as functions and `@PublicEvolving` operators
> in
> >
> > > any place in the job graph. It's not limited to only the sinks. For
> >
> > > example users could easily implement the `AsynFunction` (for
> >
> > > `AsyncWaitOperator`) that is using 2pc based on the
> `CheckpointListener`
> >
> > > interface. I'm not saying it's common, probably just a tiny minority of
> >
> > > users are doing that (if any at all), but nevertheless that's possible
> > and
> >
> > > currently (implicitly?) supported in Flink.
> >
> > >
> >
> > > Next complication is the support of bounded streams (`BoundedOneInput`
> or
> >
> > > `BoundedMultiInput` interfaces) and the closing/shutdown procedure of
> the
> >
> > > operators. Currently it works as follows:
> >
> > > 0. Task receives EndOfPartitionEvent (or source finishes)
> >
> > > 1. `endOfInput` is called on the first operator in the chain
> >
> > > 2. We quiesce the processing timers
> >
> > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for the
> > first
> >
> > > operator, so no new timers will be triggered
> >
> > > 3. We wait for the already fired timers to finish executing (spinning
> >
> > > mailbox loop)
> >
> > > 4. We are closing the first operator
> >
> > > 5. We go to the next (second) operator in the chain and repeat the
> steps
> > 1.
> >
> > > to 5.
> >
> > >
> >
> > > This is because operators can emit data after processing `endOfInput`,
> > from
> >
> > > timers, async mailbox actions and inside the `close` method itself.
> >
> > >
> >
> > > Now the problem is to support the final checkpoint with 2pc, we need
> >
> > > trigger `snapshotState` and `notifyCheckpointComplete` call at the very
> >
> > > least only after `endOfInput` call on the operator. Probably the best
> > place
> >
> > > would be in between steps 3. and 4. However that means, we would be
> > forced
> >
> > > to wait for steps 1. to 3. to finish, then wait for a next checkpoint
> to
> >
> > > trigger AND complete, before finally closing the head operator, and
> only
> >
> > > then we can start closing the next operator in the chain:
> >
> > >
> >
> > > 0. Task receives EndOfPartitionEvent (or source finishes)
> >
> > > 1. `endOfInput` is called on the first operator in the chain
> >
> > > 2. We quiesce the processing timers
> >
> > > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for the
> > first
> >
> > > operator, so no new timers will be triggered
> >
> > > 3. We wait for the already fired timers to finish executing (spinning
> >
> > > mailbox loop)
> >
> > > *3b. We wait for one more checkpoint to trigger and for the
> >
> > > `notifyCheckpointComplete` RPC.*
> >
> > > 4. We are closing the first operator
> >
> > > 5. We go to the next (second) operator in the chain and repeat the
> steps
> > 1.
> >
> > > to 5.
> >
> > >
> >
> > > That means, we can close one operator per successful checkpoint. To
> close
> >
> > > 10 operators, we would need 10 successful checkpoints.
> >
> > >
> >
> > > I was thinking about different approaches to this problem, and I
> couldn't
> >
> > > find any viable ones. All I could think of would break the current
> >
> > > `@Public` API and/or would be ugly/confusing for the users.
> >
> > >
> >
> > > For example a relatively simple solution, to introduce a `preClose` or
> >
> > > `flush` method to the operators, with a contract that after
> >
> > > `flush`, operators would be forbidden from emitting more records, so
> that
> >
> > > we can replace step 4. with this `flush` call, and then having a single
> >
> > > checkpoint to finish 2pc for all of the operators inside the chain,
> > doesn't
> >
> > > work. Sheer fact of adding this `flush` method and changing the
> contract
> >
> > > would break the current API and Yun Gao has pointed out to me, that we
> >
> > > either already support, or want to support operators that are emitting
> >
> > > records from within the `notifyCheckpointComplete` call:
> >
> > >
> >
> > > > Yun Gao:
> >
> > > > like with the new sink api there might be writer -> committer ->
> global
> >
> > > committer, the committer would need to wait for the last checkpoint to
> >
> > > commit
> >
> > > > the last piece of data, and after that it also need to emit the list
> of
> >
> > > transactions get committed to global committer to do some finalization
> >
> > > logic.
> >
> > >
> >
> > > So it wouldn't solve the problem (at least not fully).
> >
> > >
> >
> > > I don't know if anyone has any better ideas how to solve this problem?
> >
> > >
> >
> > > Piotrek
> >
> > >
> >
> > > pt., 15 sty 2021 o 14:57 Yun Gao
> >
> > > napisał(a):
> >
> > >
> >
> > > > Hi Aljoscha,
> >
> > > >
> >
> > > > I think so since we seems to do not have other divergence and new
> >
> > > > objections now. I'll open the vote then. Very thanks!
> >
> > > >
> >
> > > > Best,
> >
> > > > Yun
> >
> > > >
> >
> > > >
> >
> > > > ------------------------------------------------------------------
> >
> > > > From:Aljoscha Krettek
> >
> > > > Send Time:2021 Jan. 15 (Fri.) 21:24
> >
> > > > To:dev
> >
> > > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> > Finished
> >
> > > >
> >
> > > > Thanks for the summary! I think we can now move towards a [VOTE]
> > thread,
> >
> > > > right?
> >
> > > >
> >
> > > > On 2021/01/15 13:43, Yun Gao wrote:
> >
> > > > >1) For the problem that the "new" root task coincidently finished
> >
> > > > >before getting triggered successfully, we have listed two options in
> >
> > > > >the FLIP-147[1], for the first version, now we are not tend to go
> with
> >
> > > > >the first option that JM would re-compute and re-trigger new sources
> >
> > > > >when it realized some tasks are not triggered successfully. This
> > option
> >
> > > > >would avoid the complexity of adding new PRC and duplicating task
> >
> > > > >states, and in average case it would not cause too much overhead.
> >
> > > >
> >
> > > > You wrote "we are *not* tend to go with the first option", but I
> think
> >
> > > > you meant wo write "we tend to *now* go with the first option",
> right?
> >
> > > > That's also how it is in the FLIP, I just wanted to clarify for the
> >
> > > > mailing list.
> >
> > > >
> >
> > > >
> >
> > >
> >
> >
>

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

Posted by Guowei Ma <gu...@gmail.com>.
Thanks Yun for the detailed explanation.
A simple supplementary explanation about the sink case: Maybe we could use
`OperatorCoordinator` to avoid sending the element to the downstream
operator.
But I agree we could not limit the users not to emit records in the
`notiyCheckpointComplete`.

Best,
Guowei


On Tue, Feb 16, 2021 at 2:06 PM Yun Gao <yu...@aliyun.com.invalid>
wrote:

> Hi all,
>
> I'd like to first detail the issue with emitting records in
> notifyCheckpointComplete for context. For specific usage,
> an example would be for sink, it might want to write some metadata after
> all the transactions are committed
> (like write a marker file _SUCCESS to the output directory). This case is
> currently supported via the two level
> committers of the new sink API: when received endOfInput(), the Committer
> wait for another checkpoint to
> commits all the pending transactions and emit the list of files to the
> GlobalCommitter. The GlobalCommitter
> would wait for another checkpoint to also write the metadata with 2pc
> (Although sometimes 2pc is not needed
> for writing metadata, it should be only an optimization and still requires
> the Committer do commit before
> notifying the global Committer. Also another note is GlobalCommitter is
> also added for some other cases
> like some sinks want an commiter with dop = 1, like IceBergSink).
>
> However, a more general issue to me is that currently we do not limit
> users to not emit records in
> notifyCheckpointComplete in the API level. The sink case could be viewed
> as a special case, but in addition
> to this one, logically users could also implement their own cases that
> emits records in notifyCheckpointComplete.
>
> Best,
> Yun
>
>  ------------------Original Mail ------------------
> Sender:Arvid Heise <ar...@apache.org>
> Send Date:Fri Feb 12 20:46:04 2021
> Recipients:dev <de...@flink.apache.org>
> CC:Yun Gao <yu...@aliyun.com>
> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
> Hi Piotr,
>
>
>
> Thank you for raising your concern. Unfortunately, I do not have a better
>
> idea than doing closing of operators intermittently with checkpoints (=
>
> multiple last checkpoints).
>
>
>
> However, two ideas on how to improve the overall user experience:
>
> 1. If an operator is not relying on notifyCheckpointComplete, we can close
>
> it faster (without waiting for a checkpoint). In general, I'd assume that
>
> almost all non-sinks behave that way.
>
> 2. We may increase the checkpointing frequency for the last checkpoints. We
>
> need to avoid overloading checkpoint storages and task managers, but I
>
> assume the more operators are closed, the lower the checkpointing interval
>
> can be.
>
>
>
> For 1, I'd propose to add (name TBD):
>
>
>
> default boolean StreamOperator#requiresFinalCheckpoint() {
>
>  return true;
>
> }
>
>
>
> This means all operators are conservatively (=slowly) closed. For most
>
> operators, we can then define their behavior by overriding in
>
> AbstractUdfStreamOperator
>
>
>
> @Override
>
> boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
>
>  return userFunction instanceof CheckpointListener;
>
> }
>
>
>
> This idea can be further refined in also adding requiresFinalCheckpoint to
>
> CheckpointListener to exclude all operators with UDFs that implement
>
> CheckpointListener but do not need it for 2pc.
>
>
>
> @Override
>
> boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
>
>  return userFunction instanceof CheckpointListener &&
>
>  ((CheckpointListener) userFunction).requiresFinalCheckpoint();
>
> }
>
>
>
> That approach would also work for statebackends/snapshot strategies that
>
> require some 2pc.
>
>
>
> If we can contain it to the @PublicEvolving StreamOperator, it would be
>
> better of course.
>
>
>
> Best,
>
>
>
> Arvid
>
>
>
> On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski
>
> wrote:
>
>
>
> > Hey,
>
> >
>
> > I would like to raise a concern about implementation of the final
>
> > checkpoints taking into account operators/functions that are implementing
>
> > two phase commit (2pc) protocol for exactly-once processing with some
>
> > external state (kept outside of the Flink). Primarily exactly-once sinks.
>
> >
>
> > First of all, as I understand it, this is not planned in the first
> version
>
> > of this FLIP. I'm fine with that, however I would strongly emphasize this
>
> > in every place we will be mentioning FLIP-147 efforts. This is because
> me,
>
> > as a user, upon hearing "Flink supports checkpointing with bounded
> inputs"
>
> > I would expect 2pc to work properly and to commit the external side
> effects
>
> > upon finishing. As it is now, I (as a user) would be surprised with a
>
> > silent data loss (of not committed trailing data). This is just a remark,
>
> > that we need to attach this warning to every blog post/documentation/user
>
> > mailing list response related to "Support Checkpoints After Tasks
>
> > Finished". Also I would suggest to prioritize the follow up of supporting
>
> > 2pc.
>
> >
>
> > Secondly, I think we are missing how difficult and problematic will be
> 2pc
>
> > support with the final checkpoint.
>
> >
>
> > For starters, keep in mind that currently 2pc can be implemented by users
>
> > using both `@Public` APIs as functions and `@PublicEvolving` operators in
>
> > any place in the job graph. It's not limited to only the sinks. For
>
> > example users could easily implement the `AsynFunction` (for
>
> > `AsyncWaitOperator`) that is using 2pc based on the `CheckpointListener`
>
> > interface. I'm not saying it's common, probably just a tiny minority of
>
> > users are doing that (if any at all), but nevertheless that's possible
> and
>
> > currently (implicitly?) supported in Flink.
>
> >
>
> > Next complication is the support of bounded streams (`BoundedOneInput` or
>
> > `BoundedMultiInput` interfaces) and the closing/shutdown procedure of the
>
> > operators. Currently it works as follows:
>
> > 0. Task receives EndOfPartitionEvent (or source finishes)
>
> > 1. `endOfInput` is called on the first operator in the chain
>
> > 2. We quiesce the processing timers
>
> > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for the
> first
>
> > operator, so no new timers will be triggered
>
> > 3. We wait for the already fired timers to finish executing (spinning
>
> > mailbox loop)
>
> > 4. We are closing the first operator
>
> > 5. We go to the next (second) operator in the chain and repeat the steps
> 1.
>
> > to 5.
>
> >
>
> > This is because operators can emit data after processing `endOfInput`,
> from
>
> > timers, async mailbox actions and inside the `close` method itself.
>
> >
>
> > Now the problem is to support the final checkpoint with 2pc, we need
>
> > trigger `snapshotState` and `notifyCheckpointComplete` call at the very
>
> > least only after `endOfInput` call on the operator. Probably the best
> place
>
> > would be in between steps 3. and 4. However that means, we would be
> forced
>
> > to wait for steps 1. to 3. to finish, then wait for a next checkpoint to
>
> > trigger AND complete, before finally closing the head operator, and only
>
> > then we can start closing the next operator in the chain:
>
> >
>
> > 0. Task receives EndOfPartitionEvent (or source finishes)
>
> > 1. `endOfInput` is called on the first operator in the chain
>
> > 2. We quiesce the processing timers
>
> > (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for the
> first
>
> > operator, so no new timers will be triggered
>
> > 3. We wait for the already fired timers to finish executing (spinning
>
> > mailbox loop)
>
> > *3b. We wait for one more checkpoint to trigger and for the
>
> > `notifyCheckpointComplete` RPC.*
>
> > 4. We are closing the first operator
>
> > 5. We go to the next (second) operator in the chain and repeat the steps
> 1.
>
> > to 5.
>
> >
>
> > That means, we can close one operator per successful checkpoint. To close
>
> > 10 operators, we would need 10 successful checkpoints.
>
> >
>
> > I was thinking about different approaches to this problem, and I couldn't
>
> > find any viable ones. All I could think of would break the current
>
> > `@Public` API and/or would be ugly/confusing for the users.
>
> >
>
> > For example a relatively simple solution, to introduce a `preClose` or
>
> > `flush` method to the operators, with a contract that after
>
> > `flush`, operators would be forbidden from emitting more records, so that
>
> > we can replace step 4. with this `flush` call, and then having a single
>
> > checkpoint to finish 2pc for all of the operators inside the chain,
> doesn't
>
> > work. Sheer fact of adding this `flush` method and changing the contract
>
> > would break the current API and Yun Gao has pointed out to me, that we
>
> > either already support, or want to support operators that are emitting
>
> > records from within the `notifyCheckpointComplete` call:
>
> >
>
> > > Yun Gao:
>
> > > like with the new sink api there might be writer -> committer -> global
>
> > committer, the committer would need to wait for the last checkpoint to
>
> > commit
>
> > > the last piece of data, and after that it also need to emit the list of
>
> > transactions get committed to global committer to do some finalization
>
> > logic.
>
> >
>
> > So it wouldn't solve the problem (at least not fully).
>
> >
>
> > I don't know if anyone has any better ideas how to solve this problem?
>
> >
>
> > Piotrek
>
> >
>
> > pt., 15 sty 2021 o 14:57 Yun Gao
>
> > napisał(a):
>
> >
>
> > > Hi Aljoscha,
>
> > >
>
> > > I think so since we seems to do not have other divergence and new
>
> > > objections now. I'll open the vote then. Very thanks!
>
> > >
>
> > > Best,
>
> > > Yun
>
> > >
>
> > >
>
> > > ------------------------------------------------------------------
>
> > > From:Aljoscha Krettek
>
> > > Send Time:2021 Jan. 15 (Fri.) 21:24
>
> > > To:dev
>
> > > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks
> Finished
>
> > >
>
> > > Thanks for the summary! I think we can now move towards a [VOTE]
> thread,
>
> > > right?
>
> > >
>
> > > On 2021/01/15 13:43, Yun Gao wrote:
>
> > > >1) For the problem that the "new" root task coincidently finished
>
> > > >before getting triggered successfully, we have listed two options in
>
> > > >the FLIP-147[1], for the first version, now we are not tend to go with
>
> > > >the first option that JM would re-compute and re-trigger new sources
>
> > > >when it realized some tasks are not triggered successfully. This
> option
>
> > > >would avoid the complexity of adding new PRC and duplicating task
>
> > > >states, and in average case it would not cause too much overhead.
>
> > >
>
> > > You wrote "we are *not* tend to go with the first option", but I think
>
> > > you meant wo write "we tend to *now* go with the first option", right?
>
> > > That's also how it is in the FLIP, I just wanted to clarify for the
>
> > > mailing list.
>
> > >
>
> > >
>
> >
>
>

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi all, 

I'd like to first detail the issue with emitting records in notifyCheckpointComplete for context. For specific usage, 
an example would be for sink, it might want to write some metadata after all the transactions are committed
(like write a marker file _SUCCESS to the output directory). This case is currently supported via the two level
committers of the new sink API: when received endOfInput(), the Committer wait for another checkpoint to 
commits all the pending transactions and emit the list of files to the GlobalCommitter. The GlobalCommitter 
would wait for another checkpoint to also write the metadata with 2pc (Although sometimes 2pc is not needed 
for writing metadata, it should be only an optimization and still requires the Committer do commit before 
notifying the global Committer. Also another note is GlobalCommitter is also added for some other cases 
like some sinks want an commiter with dop = 1, like IceBergSink).

However, a more general issue to me is that currently we do not limit users to not emit records in 
notifyCheckpointComplete in the API level. The sink case could be viewed as a special case, but in addition 
to this one, logically users could also implement their own cases that emits records in notifyCheckpointComplete.

Best,
Yun

 ------------------Original Mail ------------------
Sender:Arvid Heise <ar...@apache.org>
Send Date:Fri Feb 12 20:46:04 2021
Recipients:dev <de...@flink.apache.org>
CC:Yun Gao <yu...@aliyun.com>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
Hi Piotr,



Thank you for raising your concern. Unfortunately, I do not have a better

idea than doing closing of operators intermittently with checkpoints (=

multiple last checkpoints).



However, two ideas on how to improve the overall user experience:

1. If an operator is not relying on notifyCheckpointComplete, we can close

it faster (without waiting for a checkpoint). In general, I'd assume that

almost all non-sinks behave that way.

2. We may increase the checkpointing frequency for the last checkpoints. We

need to avoid overloading checkpoint storages and task managers, but I

assume the more operators are closed, the lower the checkpointing interval

can be.



For 1, I'd propose to add (name TBD):



default boolean StreamOperator#requiresFinalCheckpoint() {

 return true;

}



This means all operators are conservatively (=slowly) closed. For most

operators, we can then define their behavior by overriding in

AbstractUdfStreamOperator



@Override

boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {

 return userFunction instanceof CheckpointListener;

}



This idea can be further refined in also adding requiresFinalCheckpoint to

CheckpointListener to exclude all operators with UDFs that implement

CheckpointListener but do not need it for 2pc.



@Override

boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {

 return userFunction instanceof CheckpointListener &&

 ((CheckpointListener) userFunction).requiresFinalCheckpoint();

}



That approach would also work for statebackends/snapshot strategies that

require some 2pc.



If we can contain it to the @PublicEvolving StreamOperator, it would be

better of course.



Best,



Arvid



On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski 

wrote:



> Hey,

>

> I would like to raise a concern about implementation of the final

> checkpoints taking into account operators/functions that are implementing

> two phase commit (2pc) protocol for exactly-once processing with some

> external state (kept outside of the Flink). Primarily exactly-once sinks.

>

> First of all, as I understand it, this is not planned in the first version

> of this FLIP. I'm fine with that, however I would strongly emphasize this

> in every place we will be mentioning FLIP-147 efforts. This is because me,

> as a user, upon hearing "Flink supports checkpointing with bounded inputs"

> I would expect 2pc to work properly and to commit the external side effects

> upon finishing. As it is now, I (as a user) would be surprised with a

> silent data loss (of not committed trailing data). This is just a remark,

> that we need to attach this warning to every blog post/documentation/user

> mailing list response related to "Support Checkpoints After Tasks

> Finished". Also I would suggest to prioritize the follow up of supporting

> 2pc.

>

> Secondly, I think we are missing how difficult and problematic will be 2pc

> support with the final checkpoint.

>

> For starters, keep in mind that currently 2pc can be implemented by users

> using both `@Public` APIs as functions and `@PublicEvolving` operators in

> any place in the job graph. It's not limited to only the sinks. For

> example users could easily implement the `AsynFunction` (for

> `AsyncWaitOperator`) that is using 2pc based on the `CheckpointListener`

> interface. I'm not saying it's common, probably just a tiny minority of

> users are doing that (if any at all), but nevertheless that's possible and

> currently (implicitly?) supported in Flink.

>

> Next complication is the support of bounded streams (`BoundedOneInput` or

> `BoundedMultiInput` interfaces) and the closing/shutdown procedure of the

> operators. Currently it works as follows:

> 0. Task receives EndOfPartitionEvent (or source finishes)

> 1. `endOfInput` is called on the first operator in the chain

> 2. We quiesce the processing timers

> (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for the first

> operator, so no new timers will be triggered

> 3. We wait for the already fired timers to finish executing (spinning

> mailbox loop)

> 4. We are closing the first operator

> 5. We go to the next (second) operator in the chain and repeat the steps 1.

> to 5.

>

> This is because operators can emit data after processing `endOfInput`, from

> timers, async mailbox actions and inside the `close` method itself.

>

> Now the problem is to support the final checkpoint with 2pc, we need

> trigger `snapshotState` and `notifyCheckpointComplete` call at the very

> least only after `endOfInput` call on the operator. Probably the best place

> would be in between steps 3. and 4. However that means, we would be forced

> to wait for steps 1. to 3. to finish, then wait for a next checkpoint to

> trigger AND complete, before finally closing the head operator, and only

> then we can start closing the next operator in the chain:

>

> 0. Task receives EndOfPartitionEvent (or source finishes)

> 1. `endOfInput` is called on the first operator in the chain

> 2. We quiesce the processing timers

> (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for the first

> operator, so no new timers will be triggered

> 3. We wait for the already fired timers to finish executing (spinning

> mailbox loop)

> *3b. We wait for one more checkpoint to trigger and for the

> `notifyCheckpointComplete` RPC.*

> 4. We are closing the first operator

> 5. We go to the next (second) operator in the chain and repeat the steps 1.

> to 5.

>

> That means, we can close one operator per successful checkpoint. To close

> 10 operators, we would need 10 successful checkpoints.

>

> I was thinking about different approaches to this problem, and I couldn't

> find any viable ones. All I could think of would break the current

> `@Public` API and/or would be ugly/confusing for the users.

>

> For example a relatively simple solution, to introduce a `preClose` or

> `flush` method to the operators, with a contract that after

> `flush`, operators would be forbidden from emitting more records, so that

> we can replace step 4. with this `flush` call, and then having a single

> checkpoint to finish 2pc for all of the operators inside the chain, doesn't

> work. Sheer fact of adding this `flush` method and changing the contract

> would break the current API and Yun Gao has pointed out to me, that we

> either already support, or want to support operators that are emitting

> records from within the `notifyCheckpointComplete` call:

>

> > Yun Gao:

> > like with the new sink api there might be writer -> committer -> global

> committer, the committer would need to wait for the last checkpoint to

> commit

> > the last piece of data, and after that it also need to emit the list of

> transactions get committed to global committer to do some finalization

> logic.

>

> So it wouldn't solve the problem (at least not fully).

>

> I don't know if anyone has any better ideas how to solve this problem?

>

> Piotrek

>

> pt., 15 sty 2021 o 14:57 Yun Gao 

> napisał(a):

>

> > Hi Aljoscha,

> >

> > I think so since we seems to do not have other divergence and new

> > objections now. I'll open the vote then. Very thanks!

> >

> > Best,

> > Yun

> >

> >

> > ------------------------------------------------------------------

> > From:Aljoscha Krettek 

> > Send Time:2021 Jan. 15 (Fri.) 21:24

> > To:dev 

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

> >

> > Thanks for the summary! I think we can now move towards a [VOTE] thread,

> > right?

> >

> > On 2021/01/15 13:43, Yun Gao wrote:

> > >1) For the problem that the "new" root task coincidently finished

> > >before getting triggered successfully, we have listed two options in

> > >the FLIP-147[1], for the first version, now we are not tend to go with

> > >the first option that JM would re-compute and re-trigger new sources

> > >when it realized some tasks are not triggered successfully. This option

> > >would avoid the complexity of adding new PRC and duplicating task

> > >states, and in average case it would not cause too much overhead.

> >

> > You wrote "we are *not* tend to go with the first option", but I think

> > you meant wo write "we tend to *now* go with the first option", right?

> > That's also how it is in the FLIP, I just wanted to clarify for the

> > mailing list.

> >

> >

>


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

Posted by Arvid Heise <ar...@apache.org>.
Hi Piotr,

Thank you for raising your concern. Unfortunately, I do not have a better
idea than doing closing of operators intermittently with checkpoints (=
multiple last checkpoints).

However, two ideas on how to improve the overall user experience:
1. If an operator is not relying on notifyCheckpointComplete, we can close
it faster (without waiting for a checkpoint). In general, I'd assume that
almost all non-sinks behave that way.
2. We may increase the checkpointing frequency for the last checkpoints. We
need to avoid overloading checkpoint storages and task managers, but I
assume the more operators are closed, the lower the checkpointing interval
can be.

For 1, I'd propose to add (name TBD):

default boolean StreamOperator#requiresFinalCheckpoint() {
    return true;
}

This means all operators are conservatively (=slowly) closed. For most
operators, we can then define their behavior by overriding in
AbstractUdfStreamOperator

@Override
boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
    return userFunction instanceof CheckpointListener;
}

This idea can be further refined in also adding requiresFinalCheckpoint to
CheckpointListener to exclude all operators with UDFs that implement
CheckpointListener but do not need it for 2pc.

@Override
boolean AbstractUdfStreamOperator#requiresFinalCheckpoint() {
    return userFunction instanceof CheckpointListener &&
            ((CheckpointListener) userFunction).requiresFinalCheckpoint();
}

That approach would also work for statebackends/snapshot strategies that
require some 2pc.

If we can contain it to the @PublicEvolving StreamOperator, it would be
better of course.

Best,

Arvid

On Fri, Feb 12, 2021 at 11:36 AM Piotr Nowojski <pn...@apache.org>
wrote:

> Hey,
>
> I would like to raise a concern about implementation of the final
> checkpoints taking into account operators/functions that are implementing
> two phase commit (2pc) protocol for exactly-once processing with some
> external state (kept outside of the Flink). Primarily exactly-once sinks.
>
> First of all, as I understand it, this is not planned in the first version
> of this FLIP. I'm fine with that, however I would strongly emphasize this
> in every place we will be mentioning FLIP-147 efforts. This is because me,
> as a user, upon hearing "Flink supports checkpointing with bounded inputs"
> I would expect 2pc to work properly and to commit the external side effects
> upon finishing. As it is now, I (as a user) would be surprised with a
> silent data loss (of not committed trailing data). This is just a remark,
> that we need to attach this warning to every blog post/documentation/user
> mailing list response related to "Support Checkpoints After Tasks
> Finished". Also I would suggest to prioritize the follow up of supporting
> 2pc.
>
> Secondly, I think we are missing how difficult and problematic will be 2pc
> support with the final checkpoint.
>
> For starters, keep in mind that currently 2pc can be implemented by users
> using both `@Public` APIs as functions and `@PublicEvolving` operators in
> any place in the job graph. It's not limited to only the sinks. For
> example users could easily implement the `AsynFunction` (for
> `AsyncWaitOperator`) that is using 2pc based on the `CheckpointListener`
> interface. I'm not saying it's common, probably just a tiny minority of
> users are doing that (if any at all), but nevertheless that's possible and
> currently (implicitly?) supported in Flink.
>
> Next complication is the support of bounded streams (`BoundedOneInput` or
> `BoundedMultiInput` interfaces) and the closing/shutdown procedure of the
> operators. Currently it works as follows:
> 0. Task receives EndOfPartitionEvent (or source finishes)
> 1. `endOfInput` is called on the first operator in the chain
> 2. We quiesce the processing timers
> (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for the first
> operator, so no new timers will be triggered
> 3. We wait for the already fired timers to finish executing (spinning
> mailbox loop)
> 4. We are closing the first operator
> 5. We go to the next (second) operator in the chain and repeat the steps 1.
> to 5.
>
> This is because operators can emit data after processing `endOfInput`, from
> timers, async mailbox actions and inside the `close` method itself.
>
> Now the problem is to support the final checkpoint with 2pc, we need
> trigger `snapshotState` and `notifyCheckpointComplete` call at the very
> least only after `endOfInput` call on the operator. Probably the best place
> would be in between steps 3. and 4. However that means, we would be forced
> to wait for steps 1. to 3. to finish, then wait for a next checkpoint to
> trigger AND complete, before finally closing the head operator, and only
> then we can start closing the next operator in the chain:
>
> 0. Task receives EndOfPartitionEvent (or source finishes)
> 1. `endOfInput` is called on the first operator in the chain
> 2. We quiesce the processing timers
> (`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for the first
> operator, so no new timers will be triggered
> 3. We wait for the already fired timers to finish executing (spinning
> mailbox loop)
> *3b. We wait for one more checkpoint to trigger and for the
> `notifyCheckpointComplete` RPC.*
> 4. We are closing the first operator
> 5. We go to the next (second) operator in the chain and repeat the steps 1.
> to 5.
>
> That means, we can close one operator per successful checkpoint. To close
> 10 operators, we would need 10 successful checkpoints.
>
> I was thinking about different approaches to this problem, and I couldn't
> find any viable ones. All I could think of would break the current
> `@Public` API and/or would be ugly/confusing for the users.
>
> For example a relatively simple solution, to introduce a `preClose` or
> `flush` method to the operators, with a contract that after
> `flush`, operators would be forbidden from emitting more records, so that
> we can replace step 4. with this `flush` call, and then having a single
> checkpoint to finish 2pc for all of the operators inside the chain, doesn't
> work. Sheer fact of adding this `flush` method and changing the contract
> would break the current API and Yun Gao has pointed out to me, that we
> either already support, or want to support operators that are emitting
> records from within the `notifyCheckpointComplete` call:
>
> > Yun Gao:
> > like with the new sink api there might be writer -> committer -> global
> committer, the committer would need to wait for the last checkpoint to
> commit
> > the last piece of data, and after that it also need to emit the list of
> transactions get committed to global committer to do some finalization
> logic.
>
> So it wouldn't solve the problem (at least not fully).
>
> I don't know if anyone has any better ideas how to solve this problem?
>
> Piotrek
>
> pt., 15 sty 2021 o 14:57 Yun Gao <yu...@aliyun.com.invalid>
> napisał(a):
>
> > Hi Aljoscha,
> >
> >     I think so since we seems to do not have other divergence and new
> > objections now. I'll open the vote then. Very thanks!
> >
> > Best,
> >  Yun
> >
> >
> > ------------------------------------------------------------------
> > From:Aljoscha Krettek <al...@apache.org>
> > Send Time:2021 Jan. 15 (Fri.) 21:24
> > To:dev <de...@flink.apache.org>
> > Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
> >
> > Thanks for the summary! I think we can now move towards a [VOTE] thread,
> > right?
> >
> > On 2021/01/15 13:43, Yun Gao wrote:
> > >1) For the problem that the "new" root task coincidently finished
> > >before getting triggered successfully, we have listed two options in
> > >the FLIP-147[1], for the first version, now we are not tend to go with
> > >the first option that JM would re-compute and re-trigger new sources
> > >when it realized some tasks are not triggered successfully. This option
> > >would avoid the complexity of adding new PRC and duplicating task
> > >states, and in average case it would not cause too much overhead.
> >
> > You wrote "we are *not* tend to go with the first option", but I think
> > you meant wo write "we tend to *now* go with the first option", right?
> > That's also how it is in the FLIP, I just wanted to clarify for the
> > mailing list.
> >
> >
>

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

Posted by Piotr Nowojski <pn...@apache.org>.
Hey,

I would like to raise a concern about implementation of the final
checkpoints taking into account operators/functions that are implementing
two phase commit (2pc) protocol for exactly-once processing with some
external state (kept outside of the Flink). Primarily exactly-once sinks.

First of all, as I understand it, this is not planned in the first version
of this FLIP. I'm fine with that, however I would strongly emphasize this
in every place we will be mentioning FLIP-147 efforts. This is because me,
as a user, upon hearing "Flink supports checkpointing with bounded inputs"
I would expect 2pc to work properly and to commit the external side effects
upon finishing. As it is now, I (as a user) would be surprised with a
silent data loss (of not committed trailing data). This is just a remark,
that we need to attach this warning to every blog post/documentation/user
mailing list response related to "Support Checkpoints After Tasks
Finished". Also I would suggest to prioritize the follow up of supporting
2pc.

Secondly, I think we are missing how difficult and problematic will be 2pc
support with the final checkpoint.

For starters, keep in mind that currently 2pc can be implemented by users
using both `@Public` APIs as functions and `@PublicEvolving` operators in
any place in the job graph. It's not limited to only the sinks. For
example users could easily implement the `AsynFunction` (for
`AsyncWaitOperator`) that is using 2pc based on the `CheckpointListener`
interface. I'm not saying it's common, probably just a tiny minority of
users are doing that (if any at all), but nevertheless that's possible and
currently (implicitly?) supported in Flink.

Next complication is the support of bounded streams (`BoundedOneInput` or
`BoundedMultiInput` interfaces) and the closing/shutdown procedure of the
operators. Currently it works as follows:
0. Task receives EndOfPartitionEvent (or source finishes)
1. `endOfInput` is called on the first operator in the chain
2. We quiesce the processing timers
(`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for the first
operator, so no new timers will be triggered
3. We wait for the already fired timers to finish executing (spinning
mailbox loop)
4. We are closing the first operator
5. We go to the next (second) operator in the chain and repeat the steps 1.
to 5.

This is because operators can emit data after processing `endOfInput`, from
timers, async mailbox actions and inside the `close` method itself.

Now the problem is to support the final checkpoint with 2pc, we need
trigger `snapshotState` and `notifyCheckpointComplete` call at the very
least only after `endOfInput` call on the operator. Probably the best place
would be in between steps 3. and 4. However that means, we would be forced
to wait for steps 1. to 3. to finish, then wait for a next checkpoint to
trigger AND complete, before finally closing the head operator, and only
then we can start closing the next operator in the chain:

0. Task receives EndOfPartitionEvent (or source finishes)
1. `endOfInput` is called on the first operator in the chain
2. We quiesce the processing timers
(`StreamOperatorWrapper#quiesceTimeServiceAndCloseOperator`) for the first
operator, so no new timers will be triggered
3. We wait for the already fired timers to finish executing (spinning
mailbox loop)
*3b. We wait for one more checkpoint to trigger and for the
`notifyCheckpointComplete` RPC.*
4. We are closing the first operator
5. We go to the next (second) operator in the chain and repeat the steps 1.
to 5.

That means, we can close one operator per successful checkpoint. To close
10 operators, we would need 10 successful checkpoints.

I was thinking about different approaches to this problem, and I couldn't
find any viable ones. All I could think of would break the current
`@Public` API and/or would be ugly/confusing for the users.

For example a relatively simple solution, to introduce a `preClose` or
`flush` method to the operators, with a contract that after
`flush`, operators would be forbidden from emitting more records, so that
we can replace step 4. with this `flush` call, and then having a single
checkpoint to finish 2pc for all of the operators inside the chain, doesn't
work. Sheer fact of adding this `flush` method and changing the contract
would break the current API and Yun Gao has pointed out to me, that we
either already support, or want to support operators that are emitting
records from within the `notifyCheckpointComplete` call:

> Yun Gao:
> like with the new sink api there might be writer -> committer -> global
committer, the committer would need to wait for the last checkpoint to
commit
> the last piece of data, and after that it also need to emit the list of
transactions get committed to global committer to do some finalization
logic.

So it wouldn't solve the problem (at least not fully).

I don't know if anyone has any better ideas how to solve this problem?

Piotrek

pt., 15 sty 2021 o 14:57 Yun Gao <yu...@aliyun.com.invalid> napisał(a):

> Hi Aljoscha,
>
>     I think so since we seems to do not have other divergence and new
> objections now. I'll open the vote then. Very thanks!
>
> Best,
>  Yun
>
>
> ------------------------------------------------------------------
> From:Aljoscha Krettek <al...@apache.org>
> Send Time:2021 Jan. 15 (Fri.) 21:24
> To:dev <de...@flink.apache.org>
> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
> Thanks for the summary! I think we can now move towards a [VOTE] thread,
> right?
>
> On 2021/01/15 13:43, Yun Gao wrote:
> >1) For the problem that the "new" root task coincidently finished
> >before getting triggered successfully, we have listed two options in
> >the FLIP-147[1], for the first version, now we are not tend to go with
> >the first option that JM would re-compute and re-trigger new sources
> >when it realized some tasks are not triggered successfully. This option
> >would avoid the complexity of adding new PRC and duplicating task
> >states, and in average case it would not cause too much overhead.
>
> You wrote "we are *not* tend to go with the first option", but I think
> you meant wo write "we tend to *now* go with the first option", right?
> That's also how it is in the FLIP, I just wanted to clarify for the
> mailing list.
>
>

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi Aljoscha,

    I think so since we seems to do not have other divergence and new objections now. I'll open the vote then. Very thanks!

Best,
 Yun


------------------------------------------------------------------
From:Aljoscha Krettek <al...@apache.org>
Send Time:2021 Jan. 15 (Fri.) 21:24
To:dev <de...@flink.apache.org>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Thanks for the summary! I think we can now move towards a [VOTE] thread, 
right?

On 2021/01/15 13:43, Yun Gao wrote:
>1) For the problem that the "new" root task coincidently finished 
>before getting triggered successfully, we have listed two options in 
>the FLIP-147[1], for the first version, now we are not tend to go with 
>the first option that JM would re-compute and re-trigger new sources 
>when it realized some tasks are not triggered successfully. This option 
>would avoid the complexity of adding new PRC and duplicating task 
>states, and in average case it would not cause too much overhead.

You wrote "we are *not* tend to go with the first option", but I think 
you meant wo write "we tend to *now* go with the first option", right?  
That's also how it is in the FLIP, I just wanted to clarify for the 
mailing list.


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

Posted by Aljoscha Krettek <al...@apache.org>.
Thanks for the summary! I think we can now move towards a [VOTE] thread, 
right?

On 2021/01/15 13:43, Yun Gao wrote:
>1) For the problem that the "new" root task coincidently finished 
>before getting triggered successfully, we have listed two options in 
>the FLIP-147[1], for the first version, now we are not tend to go with 
>the first option that JM would re-compute and re-trigger new sources 
>when it realized some tasks are not triggered successfully. This option 
>would avoid the complexity of adding new PRC and duplicating task 
>states, and in average case it would not cause too much overhead.

You wrote "we are *not* tend to go with the first option", but I think 
you meant wo write "we tend to *now* go with the first option", right?  
That's also how it is in the FLIP, I just wanted to clarify for the 
mailing list.

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
  Hi all,

We have some offline discussion together with @Arvid, @Roman and @Aljoscha and I'd 
like to post some points we discussed:

1) For the problem that the "new" root task coincidently finished before getting triggered
successfully, we have listed two options in the FLIP-147[1], for the first version, now we are not tend
to go with the first option that JM would re-compute and re-trigger new sources when it realized
some tasks are not triggered successfully. This option would avoid the complexity of adding 
new PRC and duplicating task states, and in average case it would not cause too much 
overhead.

2) For how to support operators like Sink Committer to wait for one complete checkpoint 
before exit, it would be more an issue of how to use the checkpoints after tasks finished instead 
of how to achieve checkpoint after tasks finished, thus we would like to not include this part 
first in the current discussion. We would discuss and solve this issue separately after FLIP-147 is done.

Best,
 Yun


[1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-TriggeringCheckpointsAfterTasksFinished
------------------------------------------------------------------
From:Yun Gao <yu...@aliyun.com>
Send Time:2021 Jan. 13 (Wed.) 16:09
To:dev <de...@flink.apache.org>; user <us...@flink.apache.org>
Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

  Hi all,
I updated the FLIP[1] to reflect the major discussed points in the ML thread:

 1) For the "new" root tasks finished before it received trigger message, previously we proposed 
to let JM re-compute and re-trigger the descendant tasks, but after the discussion we realized that 
it might cause overhead to JobMaster on cascade finish and large parallelism cases. Another option
might be let the StreamTask do one synchronization with the CheckpointCoordinator before get finished 
to be aware of the missed pending checkpoints, since at then EndOfPartitions are not emitted yet, it
could still broadcast barriers to its descendant tasks. I updated the details in this section[2] in the
FLIP.

2) For the barrier alignment, now we change to insert faked barriers in the input channels to avoid
interference with checkpoint alignment algorithms. One remaining issue is that for unaligned checkpoint
mode we could not snapshot the upstream tasks' result partition if it have been finished. One option
to address this issue is to make the upstream tasks to wait for buffers get flushed before exit, and 
we would include this in the future versions. I updated this part in this section[3] in the FLIP.

3) Some operators like Sink Committer need to wait for one complete checkpoint before exit. To support
the operators that need to wait for some finalization condition like the Sink committer and Async I/O, we 
could introduce a new interface to mark this kind of operators, and let the runtime to wait till the operators
reached its condition. I updated this part in this section[4] in the FLIP.

Could you have another look of the FLIP and the pending issues ? Any feedbacks are warmly welcomed 
and appreciated. Very thanks!

Best,
 Yun

[1] https://cwiki.apache.org/confluence/x/mw-ZCQ
[2] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-TriggeringCheckpointsAfterTasksFinished
[3] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-ExtendthetaskBarrierAlignment
[4] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-SupportOperatorsWaitingForCheckpointCompleteBeforeFinish


------------------------------------------------------------------
From:Yun Gao <yu...@aliyun.com.INVALID>
Send Time:2021 Jan. 12 (Tue.) 10:30
To:Khachatryan Roman <kh...@gmail.com>
Cc:dev <de...@flink.apache.org>; user <us...@flink.apache.org>
Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

    Hi Roman, 

         Very thanks for the feedbacks and suggestions!

        > I think UC will be the common case with multiple sources each with DoP > 1.
        > IIUC, waiting for EoP will be needed on each subtask each time one of it's source subtask finishes.

        Yes, waiting for EoP would be required for each input channel if we do not blocking the upstream
finished task specially. 

       > Yes, but checkpoint completion notification will not be sent until all the EOPs are processed.
      The downstream tasked get triggered indeed must wait for received EoPs from all the input channels,
I initially compared it with the completely aligned cases and now the remaining execution graph after the
trigger task could still taking normal unaligned checkpoint (like if A -> B -> C -> D, A get finished and B get 
triggered, then B -> C -> D could still taking normal unaligned checkpoint). But still it could not limit the 
possible max delay.

    > Not all declines cause job failure, particularly CHECKPOINT_DECLINED_TASK_NOT_READY doesn't.
    Sorry for mistaken the logic here and CHECKPOINT_DECLINED_TASK_NOT_READY indeed do not cause failure.
But since after a failed checkpoint we would have to wait for the checkpoint interval for the next checkpoint, I also
agree the following option would be a better one that we try to complete each checkpoint.

>> Thus another possible option might be let the upstream task to wait till all the pending buffers in the result partition has been flushed before get to finish.
> This is what I meant by "postpone JM notification from source". Just blocking the task thread wouldn't add much complexity, though I'm not sure if it would cause any problems.

>> do you think it would be ok for us to view it as an optimization and postpone it to future versions ? 
> I think that's a good idea.

 And also very sorry for here I should wrongly understand the proposals, and currently 
I also do not see explicit problems for waiting for the flush of pipeline result partition. 
Glad that we have the same viewpoints on  this issue. :) 

 Best,
  Yun



------------------------------------------------------------------
From:Khachatryan Roman <kh...@gmail.com>
Send Time:2021 Jan. 11 (Mon.) 19:14
To:Yun Gao <yu...@aliyun.com>
Cc:dev <de...@flink.apache.org>; user <us...@flink.apache.org>
Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi Yun,

> b)  With unaligned checkpoint enabled, the slower cases might happen if the downstream task processes very slowly. 
I think UC will be the common case with multiple sources each with DoP > 1.
IIUC, waiting for EoP will be needed on each subtask each time one of it's source subtask finishes.

> But since only the result partition part of the finished upstream need wait to be processed, the other part of 
> the execution graph could  still perform the unaligned checkpoint normally
Yes, but checkpoint completion notification will not be sent until all the EOPs are processed.

> Declining the RPC-trigger checkpoint would indeed simplify the implementation, but since currently by default the
> failed checkpoint would cause job failover, thus we might have some concerns in directly decline the checkpoint.
Not all declines cause job failure, particularly CHECKPOINT_DECLINED_TASK_NOT_READY doesn't.

> Thus another possible option might be let the upstream task to wait till all the pending buffers in the result partition has been flushed before get to finish.
This is what I meant by "postpone JM notification from source". Just blocking the task thread wouldn't add much complexity, though I'm not sure if it would cause any problems.

> do you think it would be ok for us to view it as an optimization and postpone it to future versions ? 
I think that's a good idea.

Regards,
Roman




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

Posted by Yun Gao <yu...@aliyun.com>.
  Hi all,

We have some offline discussion together with @Arvid, @Roman and @Aljoscha and I'd 
like to post some points we discussed:

1) For the problem that the "new" root task coincidently finished before getting triggered
successfully, we have listed two options in the FLIP-147[1], for the first version, now we are not tend
to go with the first option that JM would re-compute and re-trigger new sources when it realized
some tasks are not triggered successfully. This option would avoid the complexity of adding 
new PRC and duplicating task states, and in average case it would not cause too much 
overhead.

2) For how to support operators like Sink Committer to wait for one complete checkpoint 
before exit, it would be more an issue of how to use the checkpoints after tasks finished instead 
of how to achieve checkpoint after tasks finished, thus we would like to not include this part 
first in the current discussion. We would discuss and solve this issue separately after FLIP-147 is done.

Best,
 Yun


[1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-TriggeringCheckpointsAfterTasksFinished
------------------------------------------------------------------
From:Yun Gao <yu...@aliyun.com>
Send Time:2021 Jan. 13 (Wed.) 16:09
To:dev <de...@flink.apache.org>; user <us...@flink.apache.org>
Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

  Hi all,
I updated the FLIP[1] to reflect the major discussed points in the ML thread:

 1) For the "new" root tasks finished before it received trigger message, previously we proposed 
to let JM re-compute and re-trigger the descendant tasks, but after the discussion we realized that 
it might cause overhead to JobMaster on cascade finish and large parallelism cases. Another option
might be let the StreamTask do one synchronization with the CheckpointCoordinator before get finished 
to be aware of the missed pending checkpoints, since at then EndOfPartitions are not emitted yet, it
could still broadcast barriers to its descendant tasks. I updated the details in this section[2] in the
FLIP.

2) For the barrier alignment, now we change to insert faked barriers in the input channels to avoid
interference with checkpoint alignment algorithms. One remaining issue is that for unaligned checkpoint
mode we could not snapshot the upstream tasks' result partition if it have been finished. One option
to address this issue is to make the upstream tasks to wait for buffers get flushed before exit, and 
we would include this in the future versions. I updated this part in this section[3] in the FLIP.

3) Some operators like Sink Committer need to wait for one complete checkpoint before exit. To support
the operators that need to wait for some finalization condition like the Sink committer and Async I/O, we 
could introduce a new interface to mark this kind of operators, and let the runtime to wait till the operators
reached its condition. I updated this part in this section[4] in the FLIP.

Could you have another look of the FLIP and the pending issues ? Any feedbacks are warmly welcomed 
and appreciated. Very thanks!

Best,
 Yun

[1] https://cwiki.apache.org/confluence/x/mw-ZCQ
[2] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-TriggeringCheckpointsAfterTasksFinished
[3] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-ExtendthetaskBarrierAlignment
[4] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-SupportOperatorsWaitingForCheckpointCompleteBeforeFinish


------------------------------------------------------------------
From:Yun Gao <yu...@aliyun.com.INVALID>
Send Time:2021 Jan. 12 (Tue.) 10:30
To:Khachatryan Roman <kh...@gmail.com>
Cc:dev <de...@flink.apache.org>; user <us...@flink.apache.org>
Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

    Hi Roman, 

         Very thanks for the feedbacks and suggestions!

        > I think UC will be the common case with multiple sources each with DoP > 1.
        > IIUC, waiting for EoP will be needed on each subtask each time one of it's source subtask finishes.

        Yes, waiting for EoP would be required for each input channel if we do not blocking the upstream
finished task specially. 

       > Yes, but checkpoint completion notification will not be sent until all the EOPs are processed.
      The downstream tasked get triggered indeed must wait for received EoPs from all the input channels,
I initially compared it with the completely aligned cases and now the remaining execution graph after the
trigger task could still taking normal unaligned checkpoint (like if A -> B -> C -> D, A get finished and B get 
triggered, then B -> C -> D could still taking normal unaligned checkpoint). But still it could not limit the 
possible max delay.

    > Not all declines cause job failure, particularly CHECKPOINT_DECLINED_TASK_NOT_READY doesn't.
    Sorry for mistaken the logic here and CHECKPOINT_DECLINED_TASK_NOT_READY indeed do not cause failure.
But since after a failed checkpoint we would have to wait for the checkpoint interval for the next checkpoint, I also
agree the following option would be a better one that we try to complete each checkpoint.

>> Thus another possible option might be let the upstream task to wait till all the pending buffers in the result partition has been flushed before get to finish.
> This is what I meant by "postpone JM notification from source". Just blocking the task thread wouldn't add much complexity, though I'm not sure if it would cause any problems.

>> do you think it would be ok for us to view it as an optimization and postpone it to future versions ? 
> I think that's a good idea.

 And also very sorry for here I should wrongly understand the proposals, and currently 
I also do not see explicit problems for waiting for the flush of pipeline result partition. 
Glad that we have the same viewpoints on  this issue. :) 

 Best,
  Yun



------------------------------------------------------------------
From:Khachatryan Roman <kh...@gmail.com>
Send Time:2021 Jan. 11 (Mon.) 19:14
To:Yun Gao <yu...@aliyun.com>
Cc:dev <de...@flink.apache.org>; user <us...@flink.apache.org>
Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi Yun,

> b)  With unaligned checkpoint enabled, the slower cases might happen if the downstream task processes very slowly. 
I think UC will be the common case with multiple sources each with DoP > 1.
IIUC, waiting for EoP will be needed on each subtask each time one of it's source subtask finishes.

> But since only the result partition part of the finished upstream need wait to be processed, the other part of 
> the execution graph could  still perform the unaligned checkpoint normally
Yes, but checkpoint completion notification will not be sent until all the EOPs are processed.

> Declining the RPC-trigger checkpoint would indeed simplify the implementation, but since currently by default the
> failed checkpoint would cause job failover, thus we might have some concerns in directly decline the checkpoint.
Not all declines cause job failure, particularly CHECKPOINT_DECLINED_TASK_NOT_READY doesn't.

> Thus another possible option might be let the upstream task to wait till all the pending buffers in the result partition has been flushed before get to finish.
This is what I meant by "postpone JM notification from source". Just blocking the task thread wouldn't add much complexity, though I'm not sure if it would cause any problems.

> do you think it would be ok for us to view it as an optimization and postpone it to future versions ? 
I think that's a good idea.

Regards,
Roman




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

Posted by Yun Gao <yu...@aliyun.com>.
  Hi all,
I updated the FLIP[1] to reflect the major discussed points in the ML thread:

 1) For the "new" root tasks finished before it received trigger message, previously we proposed 
to let JM re-compute and re-trigger the descendant tasks, but after the discussion we realized that 
it might cause overhead to JobMaster on cascade finish and large parallelism cases. Another option
might be let the StreamTask do one synchronization with the CheckpointCoordinator before get finished 
to be aware of the missed pending checkpoints, since at then EndOfPartitions are not emitted yet, it
could still broadcast barriers to its descendant tasks. I updated the details in this section[2] in the
FLIP.

2) For the barrier alignment, now we change to insert faked barriers in the input channels to avoid
interference with checkpoint alignment algorithms. One remaining issue is that for unaligned checkpoint
mode we could not snapshot the upstream tasks' result partition if it have been finished. One option
to address this issue is to make the upstream tasks to wait for buffers get flushed before exit, and 
we would include this in the future versions. I updated this part in this section[3] in the FLIP.

3) Some operators like Sink Committer need to wait for one complete checkpoint before exit. To support
the operators that need to wait for some finalization condition like the Sink committer and Async I/O, we 
could introduce a new interface to mark this kind of operators, and let the runtime to wait till the operators
reached its condition. I updated this part in this section[4] in the FLIP.

Could you have another look of the FLIP and the pending issues ? Any feedbacks are warmly welcomed 
and appreciated. Very thanks!

Best,
 Yun

[1] https://cwiki.apache.org/confluence/x/mw-ZCQ
[2] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-TriggeringCheckpointsAfterTasksFinished
[3] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-ExtendthetaskBarrierAlignment
[4] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-SupportOperatorsWaitingForCheckpointCompleteBeforeFinish


------------------------------------------------------------------
From:Yun Gao <yu...@aliyun.com.INVALID>
Send Time:2021 Jan. 12 (Tue.) 10:30
To:Khachatryan Roman <kh...@gmail.com>
Cc:dev <de...@flink.apache.org>; user <us...@flink.apache.org>
Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

    Hi Roman, 

         Very thanks for the feedbacks and suggestions!

        > I think UC will be the common case with multiple sources each with DoP > 1.
        > IIUC, waiting for EoP will be needed on each subtask each time one of it's source subtask finishes.

        Yes, waiting for EoP would be required for each input channel if we do not blocking the upstream
finished task specially. 

       > Yes, but checkpoint completion notification will not be sent until all the EOPs are processed.
      The downstream tasked get triggered indeed must wait for received EoPs from all the input channels,
I initially compared it with the completely aligned cases and now the remaining execution graph after the
trigger task could still taking normal unaligned checkpoint (like if A -> B -> C -> D, A get finished and B get 
triggered, then B -> C -> D could still taking normal unaligned checkpoint). But still it could not limit the 
possible max delay.

    > Not all declines cause job failure, particularly CHECKPOINT_DECLINED_TASK_NOT_READY doesn't.
    Sorry for mistaken the logic here and CHECKPOINT_DECLINED_TASK_NOT_READY indeed do not cause failure.
But since after a failed checkpoint we would have to wait for the checkpoint interval for the next checkpoint, I also
agree the following option would be a better one that we try to complete each checkpoint.

>> Thus another possible option might be let the upstream task to wait till all the pending buffers in the result partition has been flushed before get to finish.
> This is what I meant by "postpone JM notification from source". Just blocking the task thread wouldn't add much complexity, though I'm not sure if it would cause any problems.

>> do you think it would be ok for us to view it as an optimization and postpone it to future versions ? 
> I think that's a good idea.

 And also very sorry for here I should wrongly understand the proposals, and currently 
I also do not see explicit problems for waiting for the flush of pipeline result partition. 
Glad that we have the same viewpoints on  this issue. :) 

 Best,
  Yun



------------------------------------------------------------------
From:Khachatryan Roman <kh...@gmail.com>
Send Time:2021 Jan. 11 (Mon.) 19:14
To:Yun Gao <yu...@aliyun.com>
Cc:dev <de...@flink.apache.org>; user <us...@flink.apache.org>
Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi Yun,

> b)  With unaligned checkpoint enabled, the slower cases might happen if the downstream task processes very slowly. 
I think UC will be the common case with multiple sources each with DoP > 1.
IIUC, waiting for EoP will be needed on each subtask each time one of it's source subtask finishes.

> But since only the result partition part of the finished upstream need wait to be processed, the other part of 
> the execution graph could  still perform the unaligned checkpoint normally
Yes, but checkpoint completion notification will not be sent until all the EOPs are processed.

> Declining the RPC-trigger checkpoint would indeed simplify the implementation, but since currently by default the
> failed checkpoint would cause job failover, thus we might have some concerns in directly decline the checkpoint.
Not all declines cause job failure, particularly CHECKPOINT_DECLINED_TASK_NOT_READY doesn't.

> Thus another possible option might be let the upstream task to wait till all the pending buffers in the result partition has been flushed before get to finish.
This is what I meant by "postpone JM notification from source". Just blocking the task thread wouldn't add much complexity, though I'm not sure if it would cause any problems.

> do you think it would be ok for us to view it as an optimization and postpone it to future versions ? 
I think that's a good idea.

Regards,
Roman



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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
  Hi all,
I updated the FLIP[1] to reflect the major discussed points in the ML thread:

 1) For the "new" root tasks finished before it received trigger message, previously we proposed 
to let JM re-compute and re-trigger the descendant tasks, but after the discussion we realized that 
it might cause overhead to JobMaster on cascade finish and large parallelism cases. Another option
might be let the StreamTask do one synchronization with the CheckpointCoordinator before get finished 
to be aware of the missed pending checkpoints, since at then EndOfPartitions are not emitted yet, it
could still broadcast barriers to its descendant tasks. I updated the details in this section[2] in the
FLIP.

2) For the barrier alignment, now we change to insert faked barriers in the input channels to avoid
interference with checkpoint alignment algorithms. One remaining issue is that for unaligned checkpoint
mode we could not snapshot the upstream tasks' result partition if it have been finished. One option
to address this issue is to make the upstream tasks to wait for buffers get flushed before exit, and 
we would include this in the future versions. I updated this part in this section[3] in the FLIP.

3) Some operators like Sink Committer need to wait for one complete checkpoint before exit. To support
the operators that need to wait for some finalization condition like the Sink committer and Async I/O, we 
could introduce a new interface to mark this kind of operators, and let the runtime to wait till the operators
reached its condition. I updated this part in this section[4] in the FLIP.

Could you have another look of the FLIP and the pending issues ? Any feedbacks are warmly welcomed 
and appreciated. Very thanks!

Best,
 Yun

[1] https://cwiki.apache.org/confluence/x/mw-ZCQ
[2] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-TriggeringCheckpointsAfterTasksFinished
[3] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-ExtendthetaskBarrierAlignment
[4] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-SupportOperatorsWaitingForCheckpointCompleteBeforeFinish


------------------------------------------------------------------
From:Yun Gao <yu...@aliyun.com.INVALID>
Send Time:2021 Jan. 12 (Tue.) 10:30
To:Khachatryan Roman <kh...@gmail.com>
Cc:dev <de...@flink.apache.org>; user <us...@flink.apache.org>
Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

    Hi Roman, 

         Very thanks for the feedbacks and suggestions!

        > I think UC will be the common case with multiple sources each with DoP > 1.
        > IIUC, waiting for EoP will be needed on each subtask each time one of it's source subtask finishes.

        Yes, waiting for EoP would be required for each input channel if we do not blocking the upstream
finished task specially. 

       > Yes, but checkpoint completion notification will not be sent until all the EOPs are processed.
      The downstream tasked get triggered indeed must wait for received EoPs from all the input channels,
I initially compared it with the completely aligned cases and now the remaining execution graph after the
trigger task could still taking normal unaligned checkpoint (like if A -> B -> C -> D, A get finished and B get 
triggered, then B -> C -> D could still taking normal unaligned checkpoint). But still it could not limit the 
possible max delay.

    > Not all declines cause job failure, particularly CHECKPOINT_DECLINED_TASK_NOT_READY doesn't.
    Sorry for mistaken the logic here and CHECKPOINT_DECLINED_TASK_NOT_READY indeed do not cause failure.
But since after a failed checkpoint we would have to wait for the checkpoint interval for the next checkpoint, I also
agree the following option would be a better one that we try to complete each checkpoint.

>> Thus another possible option might be let the upstream task to wait till all the pending buffers in the result partition has been flushed before get to finish.
> This is what I meant by "postpone JM notification from source". Just blocking the task thread wouldn't add much complexity, though I'm not sure if it would cause any problems.

>> do you think it would be ok for us to view it as an optimization and postpone it to future versions ? 
> I think that's a good idea.

 And also very sorry for here I should wrongly understand the proposals, and currently 
I also do not see explicit problems for waiting for the flush of pipeline result partition. 
Glad that we have the same viewpoints on  this issue. :) 

 Best,
  Yun



------------------------------------------------------------------
From:Khachatryan Roman <kh...@gmail.com>
Send Time:2021 Jan. 11 (Mon.) 19:14
To:Yun Gao <yu...@aliyun.com>
Cc:dev <de...@flink.apache.org>; user <us...@flink.apache.org>
Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi Yun,

> b)  With unaligned checkpoint enabled, the slower cases might happen if the downstream task processes very slowly. 
I think UC will be the common case with multiple sources each with DoP > 1.
IIUC, waiting for EoP will be needed on each subtask each time one of it's source subtask finishes.

> But since only the result partition part of the finished upstream need wait to be processed, the other part of 
> the execution graph could  still perform the unaligned checkpoint normally
Yes, but checkpoint completion notification will not be sent until all the EOPs are processed.

> Declining the RPC-trigger checkpoint would indeed simplify the implementation, but since currently by default the
> failed checkpoint would cause job failover, thus we might have some concerns in directly decline the checkpoint.
Not all declines cause job failure, particularly CHECKPOINT_DECLINED_TASK_NOT_READY doesn't.

> Thus another possible option might be let the upstream task to wait till all the pending buffers in the result partition has been flushed before get to finish.
This is what I meant by "postpone JM notification from source". Just blocking the task thread wouldn't add much complexity, though I'm not sure if it would cause any problems.

> do you think it would be ok for us to view it as an optimization and postpone it to future versions ? 
I think that's a good idea.

Regards,
Roman



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

Posted by Yun Gao <yu...@aliyun.com>.
    Hi Roman, 

         Very thanks for the feedbacks and suggestions!

        > I think UC will be the common case with multiple sources each with DoP > 1.
        > IIUC, waiting for EoP will be needed on each subtask each time one of it's source subtask finishes.

        Yes, waiting for EoP would be required for each input channel if we do not blocking the upstream
finished task specially. 

       > Yes, but checkpoint completion notification will not be sent until all the EOPs are processed.
      The downstream tasked get triggered indeed must wait for received EoPs from all the input channels,
I initially compared it with the completely aligned cases and now the remaining execution graph after the
trigger task could still taking normal unaligned checkpoint (like if A -> B -> C -> D, A get finished and B get 
triggered, then B -> C -> D could still taking normal unaligned checkpoint). But still it could not limit the 
possible max delay.

    > Not all declines cause job failure, particularly CHECKPOINT_DECLINED_TASK_NOT_READY doesn't.
    Sorry for mistaken the logic here and CHECKPOINT_DECLINED_TASK_NOT_READY indeed do not cause failure.
But since after a failed checkpoint we would have to wait for the checkpoint interval for the next checkpoint, I also
agree the following option would be a better one that we try to complete each checkpoint.

>> Thus another possible option might be let the upstream task to wait till all the pending buffers in the result partition has been flushed before get to finish.
> This is what I meant by "postpone JM notification from source". Just blocking the task thread wouldn't add much complexity, though I'm not sure if it would cause any problems.

>> do you think it would be ok for us to view it as an optimization and postpone it to future versions ? 
> I think that's a good idea.

 And also very sorry for here I should wrongly understand the proposals, and currently 
I also do not see explicit problems for waiting for the flush of pipeline result partition. 
Glad that we have the same viewpoints on  this issue. :) 

 Best,
  Yun



------------------------------------------------------------------
From:Khachatryan Roman <kh...@gmail.com>
Send Time:2021 Jan. 11 (Mon.) 19:14
To:Yun Gao <yu...@aliyun.com>
Cc:dev <de...@flink.apache.org>; user <us...@flink.apache.org>
Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi Yun,

> b)  With unaligned checkpoint enabled, the slower cases might happen if the downstream task processes very slowly. 
I think UC will be the common case with multiple sources each with DoP > 1.
IIUC, waiting for EoP will be needed on each subtask each time one of it's source subtask finishes.

> But since only the result partition part of the finished upstream need wait to be processed, the other part of 
> the execution graph could  still perform the unaligned checkpoint normally
Yes, but checkpoint completion notification will not be sent until all the EOPs are processed.

> Declining the RPC-trigger checkpoint would indeed simplify the implementation, but since currently by default the
> failed checkpoint would cause job failover, thus we might have some concerns in directly decline the checkpoint.
Not all declines cause job failure, particularly CHECKPOINT_DECLINED_TASK_NOT_READY doesn't.

> Thus another possible option might be let the upstream task to wait till all the pending buffers in the result partition has been flushed before get to finish.
This is what I meant by "postpone JM notification from source". Just blocking the task thread wouldn't add much complexity, though I'm not sure if it would cause any problems.

> do you think it would be ok for us to view it as an optimization and postpone it to future versions ? 
I think that's a good idea.

Regards,
Roman


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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
    Hi Roman, 

         Very thanks for the feedbacks and suggestions!

        > I think UC will be the common case with multiple sources each with DoP > 1.
        > IIUC, waiting for EoP will be needed on each subtask each time one of it's source subtask finishes.

        Yes, waiting for EoP would be required for each input channel if we do not blocking the upstream
finished task specially. 

       > Yes, but checkpoint completion notification will not be sent until all the EOPs are processed.
      The downstream tasked get triggered indeed must wait for received EoPs from all the input channels,
I initially compared it with the completely aligned cases and now the remaining execution graph after the
trigger task could still taking normal unaligned checkpoint (like if A -> B -> C -> D, A get finished and B get 
triggered, then B -> C -> D could still taking normal unaligned checkpoint). But still it could not limit the 
possible max delay.

    > Not all declines cause job failure, particularly CHECKPOINT_DECLINED_TASK_NOT_READY doesn't.
    Sorry for mistaken the logic here and CHECKPOINT_DECLINED_TASK_NOT_READY indeed do not cause failure.
But since after a failed checkpoint we would have to wait for the checkpoint interval for the next checkpoint, I also
agree the following option would be a better one that we try to complete each checkpoint.

>> Thus another possible option might be let the upstream task to wait till all the pending buffers in the result partition has been flushed before get to finish.
> This is what I meant by "postpone JM notification from source". Just blocking the task thread wouldn't add much complexity, though I'm not sure if it would cause any problems.

>> do you think it would be ok for us to view it as an optimization and postpone it to future versions ? 
> I think that's a good idea.

 And also very sorry for here I should wrongly understand the proposals, and currently 
I also do not see explicit problems for waiting for the flush of pipeline result partition. 
Glad that we have the same viewpoints on  this issue. :) 

 Best,
  Yun



------------------------------------------------------------------
From:Khachatryan Roman <kh...@gmail.com>
Send Time:2021 Jan. 11 (Mon.) 19:14
To:Yun Gao <yu...@aliyun.com>
Cc:dev <de...@flink.apache.org>; user <us...@flink.apache.org>
Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi Yun,

> b)  With unaligned checkpoint enabled, the slower cases might happen if the downstream task processes very slowly. 
I think UC will be the common case with multiple sources each with DoP > 1.
IIUC, waiting for EoP will be needed on each subtask each time one of it's source subtask finishes.

> But since only the result partition part of the finished upstream need wait to be processed, the other part of 
> the execution graph could  still perform the unaligned checkpoint normally
Yes, but checkpoint completion notification will not be sent until all the EOPs are processed.

> Declining the RPC-trigger checkpoint would indeed simplify the implementation, but since currently by default the
> failed checkpoint would cause job failover, thus we might have some concerns in directly decline the checkpoint.
Not all declines cause job failure, particularly CHECKPOINT_DECLINED_TASK_NOT_READY doesn't.

> Thus another possible option might be let the upstream task to wait till all the pending buffers in the result partition has been flushed before get to finish.
This is what I meant by "postpone JM notification from source". Just blocking the task thread wouldn't add much complexity, though I'm not sure if it would cause any problems.

> do you think it would be ok for us to view it as an optimization and postpone it to future versions ? 
I think that's a good idea.

Regards,
Roman


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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
      Hi Roman,

          Very thanks for the feedbacks !
        > Probably it would be simpler to just decline the RPC-triggered checkpoint 
        > if not all inputs of this task are finished (with CHECKPOINT_DECLINED_TASK_NOT_READY).

        > But I wonder how significantly this waiting for EoP from every input will delay performing the first checkpoint 
        > by B after becoming a new source. This may in turn impact exactly-once sinks and incremental checkpoints.
        > Maybe a better option would be to postpone JM notification from source until it's EoP is consumed?

       I also agree with that there would indeed be possible cases that the checkpoint get slower since it could not skip
the data in  the result partition of the finished upstream task:
            a) For aligned checkpoint, the cases would not happen since the downstream tasks would always need to 
                process the buffers in order. 
           b)  With unaligned checkpoint enabled, the slower cases might happen if the downstream task processes very 
                slowly. 

       But since only the result partition part of the finished upstream need wait to be processed, the other part of 
the execution graph could  still perform the unaligned checkpoint normally, I think the average delay caused would 
 be much lower than the completely aligned checkpoint, but there would still be extremely bad cases that
       the delay is long. 

       Declining the RPC-trigger checkpoint would indeed simplify the implementation, but since currently by default the
       failed checkpoint would cause job failover, thus we might have some concerns in directly decline the checkpoint.
       For postpone the notification the JM notification, since current JM should not be able to know if the task has 
       received all the EndOfPartition from the upstream tasks, we might need to introduce new RPC for notifying the 
       state and since the triggering is not atomic, we may also met with some  synchronization issues between JM and TM, 
       which would introduce some complexity. 
      Thus another possible option might be let the upstream task to wait till all the pending buffers in the result partition has
      been flushed before get to finish. We could only do the wait for the PipelineResultPartition so it won't affect the batch
      jobs. With the waiting the unaligned checkpoint could continue to trigger the upstream task and skip the buffers in
      the result partition. Since the result partition state would be kept within the operator state of the last operator, after failover
      we would found that the last operator has an non-empty state and we would restart the tasks containing this operator to 
      resend the snapshotted buffers. Of course this would also introduce some complexity, and since the probability of long delay 
      would be lower than the completely aligned case, do you think it would be ok for us to view it as an optimization and 
      postpone it to future versions ? 

     Best,
     Yun


------------------------------------------------------------------
From:Khachatryan Roman <kh...@gmail.com>
Send Time:2021 Jan. 11 (Mon.) 05:46
To:Yun Gao <yu...@aliyun.com>
Cc:Arvid Heise <ar...@ververica.com>; dev <de...@flink.apache.org>; user <us...@flink.apache.org>
Subject:Re: Re: [DISCUSS] 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" to trigger checkpoint via RPC. For task B, if it received checkpoint trigger, it would know that all its precedant tasks
> are finished, then it would wait till all the InputChannel received EndOfPartition from the network (namely inputChannel.onBuffer() 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.

Probably it would be simpler to just decline the RPC-triggered checkpoint if not all inputs of this task are finished (with CHECKPOINT_DECLINED_TASK_NOT_READY).

But I wonder how significantly this waiting for EoP from every input will delay performing the first checkpoint by B after becoming a new source. This may in turn impact exactly-once sinks and incremental checkpoints.
Maybe a better option would be to postpone JM notification from source until it's EoP is consumed?

Regards,
Roman


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

Posted by Yun Gao <yu...@aliyun.com>.
      Hi Roman,

          Very thanks for the feedbacks !
        > Probably it would be simpler to just decline the RPC-triggered checkpoint 
        > if not all inputs of this task are finished (with CHECKPOINT_DECLINED_TASK_NOT_READY).

        > But I wonder how significantly this waiting for EoP from every input will delay performing the first checkpoint 
        > by B after becoming a new source. This may in turn impact exactly-once sinks and incremental checkpoints.
        > Maybe a better option would be to postpone JM notification from source until it's EoP is consumed?

       I also agree with that there would indeed be possible cases that the checkpoint get slower since it could not skip
the data in  the result partition of the finished upstream task:
            a) For aligned checkpoint, the cases would not happen since the downstream tasks would always need to 
                process the buffers in order. 
           b)  With unaligned checkpoint enabled, the slower cases might happen if the downstream task processes very 
                slowly. 

       But since only the result partition part of the finished upstream need wait to be processed, the other part of 
the execution graph could  still perform the unaligned checkpoint normally, I think the average delay caused would 
 be much lower than the completely aligned checkpoint, but there would still be extremely bad cases that
       the delay is long. 

       Declining the RPC-trigger checkpoint would indeed simplify the implementation, but since currently by default the
       failed checkpoint would cause job failover, thus we might have some concerns in directly decline the checkpoint.
       For postpone the notification the JM notification, since current JM should not be able to know if the task has 
       received all the EndOfPartition from the upstream tasks, we might need to introduce new RPC for notifying the 
       state and since the triggering is not atomic, we may also met with some  synchronization issues between JM and TM, 
       which would introduce some complexity. 
      Thus another possible option might be let the upstream task to wait till all the pending buffers in the result partition has
      been flushed before get to finish. We could only do the wait for the PipelineResultPartition so it won't affect the batch
      jobs. With the waiting the unaligned checkpoint could continue to trigger the upstream task and skip the buffers in
      the result partition. Since the result partition state would be kept within the operator state of the last operator, after failover
      we would found that the last operator has an non-empty state and we would restart the tasks containing this operator to 
      resend the snapshotted buffers. Of course this would also introduce some complexity, and since the probability of long delay 
      would be lower than the completely aligned case, do you think it would be ok for us to view it as an optimization and 
      postpone it to future versions ? 

     Best,
     Yun


------------------------------------------------------------------
From:Khachatryan Roman <kh...@gmail.com>
Send Time:2021 Jan. 11 (Mon.) 05:46
To:Yun Gao <yu...@aliyun.com>
Cc:Arvid Heise <ar...@ververica.com>; dev <de...@flink.apache.org>; user <us...@flink.apache.org>
Subject:Re: Re: [DISCUSS] 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" to trigger checkpoint via RPC. For task B, if it received checkpoint trigger, it would know that all its precedant tasks
> are finished, then it would wait till all the InputChannel received EndOfPartition from the network (namely inputChannel.onBuffer() 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.

Probably it would be simpler to just decline the RPC-triggered checkpoint if not all inputs of this task are finished (with CHECKPOINT_DECLINED_TASK_NOT_READY).

But I wonder how significantly this waiting for EoP from every input will delay performing the first checkpoint by B after becoming a new source. This may in turn impact exactly-once sinks and incremental checkpoints.
Maybe a better option would be to postpone JM notification from source until it's EoP is consumed?

Regards,
Roman


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

Posted by Khachatryan Roman <kh...@gmail.com>.
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" to trigger checkpoint via RPC. For task B,
if it received checkpoint trigger, it would know that all its precedant
tasks
> are finished, then it would wait till all the InputChannel received
EndOfPartition from the network (namely inputChannel.onBuffer() 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.

Probably it would be simpler to just decline the RPC-triggered checkpoint
if not all inputs of this task are finished (with
CHECKPOINT_DECLINED_TASK_NOT_READY).

But I wonder how significantly this waiting for EoP from every input will
delay performing the first checkpoint by B after becoming a new source.
This may in turn impact exactly-once sinks and incremental checkpoints.
Maybe a better option would be to postpone JM notification from source
until it's EoP is consumed?

Regards,
Roman


On Thu, Jan 7, 2021 at 5:01 PM Yun Gao <yu...@aliyun.com> wrote:

> Hi Roman,
>
>    Very thanks for the feedbacks! I'll try to answer the issues inline:
>
> > 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 short time
> until the job finishes completely.
> > And compared to other options, complexity seems much lower. Or are
> differences in task completion times so huge and so common?
>
> There might be mixed jobs with both bounded sources and unbounded sources,
> in this case, the resource for the finished
> part of the job would not be able to be released.
>
> And the Option 1 also complicates the semantics of the EndOfPartition,
> since if we holding the tasks and we still need to
> notify the following tasks about all records are sent, we would have to
> introduce some kind of pre-EndOfPartition messages,
> which is similar to the current EndOfPartition, but do not cause the
> channels to be released.
>
> > 2. I think it would be helpful to describe how is rescaling handled in
> Options 2 and 3 (or maybe it's not supported for jobs about to finish).
>
> For Option 2 and 3 we managed the states via the unit of operator, thus
> the process of rescaling would be the same with the normal checkpoint.
> For example, support one operator resides in a tasks with parallelism 4,
> if 2 fo the subtasks are finished, now the state of the operator is
> composed
> of the state of the 2 remaining subtask instance, if we rescale to 5 after
> failover, the state of the 2 previous remaining subtasks would be
> re-distributed
> to the 5 new subtasks after failover.
>
> If before failover all the 4 subtasks are finished, the operator would be
> marked as finished, after failover the operator would be still marked as
> finished,
> and all the subtask instance of this operator would skip all the methods
> like open(), endOfInput(), close() and would be excluded when taking
> checkpoints
> after failover.
>
>
> > 3. Option 3 assumes that the state of a finished task is not used.
> That's true for operator state, but what about channel state (captured by
> unaligned checkpoint)?
> > I think it still has to be sent downstream which invalidates this Option.
>
> For unaligned checkpoint, if in one checkpoint a subtask is marked as
> finished, then its descandent tasks would wait all the records are received
> from the finished tasks before taking checkpoint, thus in this case we
> would not have result partition state, but only have channel state for the
> downstream tasks that are still running.
>
> 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" to trigger checkpoint via RPC. For task B, if
> it received checkpoint trigger, it would know that all its precedant tasks
> are finished, then it would wait till all the InputChannel received
> EndOfPartition from the network (namely inputChannel.onBuffer() 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 more clear~
>
> Best,
>  Yun
>
>
> ------------------Original Mail ------------------
> *Sender:*Khachatryan Roman <kh...@gmail.com>
> *Send Date:*Thu Jan 7 21:55:52 2021
> *Recipients:*Arvid Heise <ar...@ververica.com>
> *CC:*dev <de...@flink.apache.org>, user <us...@flink.apache.org>
> *Subject:*Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
>> 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 short time until
>> the job finishes completely.
>> And compared to other options, complexity seems much lower. Or are
>> differences in task completion times so huge and so common?
>>
>> 2. I think it would be helpful to describe how is rescaling handled in
>> Options 2 and 3 (or maybe it's not supported for jobs about to finish).
>>
>> 3. Option 3 assumes that the state of a finished task is not used. That's
>> true for operator state, but what about channel state (captured by
>> unaligned checkpoint)? I think it still has to be sent downstream which
>> invalidates this Option.
>>
>> Regards,
>> Roman
>>
>>
>> On Thu, Jan 7, 2021 at 1:21 PM Arvid Heise <ar...@ververica.com> wrote:
>>
>>> 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 interface would have a method like `isFinalized()` that the
>>>> framework can call after each checkpoint (and potentially at other
>>>> points)
>>>
>>>
>>> I think we are mixing two different things here that may require
>>> different solutions:
>>> 1. Tasks (=sink) that may need to do something with the final checkpoint.
>>> 2. Tasks that only finish after having finished operations that do not
>>> depend on data flow (async I/O, but I could also think of some timer
>>> actions in process functions).
>>>
>>> Your proposal would help most for the first case. The second case can
>>> solved entirely with current methods without being especially complicated:
>>> - EOP is only emitted once Async I/O is done with all background tasks
>>> - All timers are fired in a process function (I think we rather want to
>>> fire immediately on EOP but that's a different discussion)
>>> The advantage of this approach over your idea is that you don't need to
>>> wait for a checkpoint to complete to check for finalization.
>>>
>>> Now let's look at the first case. I see two alternatives:
>>> - The new sink interface implicitly incorporates this listener. Since I
>>> don't see a use case outside sinks, we could simply add this method to the
>>> new sink interface.
>>> - We implicitly assume that a sink is done after having a successful
>>> checkpoint at the end. Then we just need a tag interface
>>> `RequiresFinalization`. It also feels like we should add the property
>>> `final` to checkpoint options to help the sink detect that this is the last
>>> checkpoint to be taken. We could also try to always have the final
>>> checkpoint without tag interface on new sinks...
>>>
>>> On Thu, Jan 7, 2021 at 11:58 AM Aljoscha Krettek <al...@apache.org>
>>> wrote:
>>>
>>>> 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 least for Async I/O and
>>>> potentially for sinks.
>>>>
>>>> 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 interface would have a method like `isFinalized()` that the
>>>> framework can call after each checkpoint (and potentially at other
>>>> points)
>>>>
>>>> This way we would decouple that logic from things that don't actually
>>>> need it. What do you think?
>>>>
>>>> Best,
>>>> Aljoscha
>>>>
>>>
>>>
>>> --
>>>
>>> Arvid Heise | Senior Java Developer
>>>
>>> <https://www.ververica.com/>
>>>
>>> Follow us @VervericaData
>>>
>>> --
>>>
>>> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
>>> Conference
>>>
>>> Stream Processing | Event Driven | Real Time
>>>
>>> --
>>>
>>> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>>>
>>> --
>>> Ververica GmbH
>>> Registered at Amtsgericht Charlottenburg: HRB 158244 B
>>> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
>>> (Toni) Cheng
>>>
>>

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

Posted by Khachatryan Roman <kh...@gmail.com>.
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" to trigger checkpoint via RPC. For task B,
if it received checkpoint trigger, it would know that all its precedant
tasks
> are finished, then it would wait till all the InputChannel received
EndOfPartition from the network (namely inputChannel.onBuffer() 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.

Probably it would be simpler to just decline the RPC-triggered checkpoint
if not all inputs of this task are finished (with
CHECKPOINT_DECLINED_TASK_NOT_READY).

But I wonder how significantly this waiting for EoP from every input will
delay performing the first checkpoint by B after becoming a new source.
This may in turn impact exactly-once sinks and incremental checkpoints.
Maybe a better option would be to postpone JM notification from source
until it's EoP is consumed?

Regards,
Roman


On Thu, Jan 7, 2021 at 5:01 PM Yun Gao <yu...@aliyun.com> wrote:

> Hi Roman,
>
>    Very thanks for the feedbacks! I'll try to answer the issues inline:
>
> > 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 short time
> until the job finishes completely.
> > And compared to other options, complexity seems much lower. Or are
> differences in task completion times so huge and so common?
>
> There might be mixed jobs with both bounded sources and unbounded sources,
> in this case, the resource for the finished
> part of the job would not be able to be released.
>
> And the Option 1 also complicates the semantics of the EndOfPartition,
> since if we holding the tasks and we still need to
> notify the following tasks about all records are sent, we would have to
> introduce some kind of pre-EndOfPartition messages,
> which is similar to the current EndOfPartition, but do not cause the
> channels to be released.
>
> > 2. I think it would be helpful to describe how is rescaling handled in
> Options 2 and 3 (or maybe it's not supported for jobs about to finish).
>
> For Option 2 and 3 we managed the states via the unit of operator, thus
> the process of rescaling would be the same with the normal checkpoint.
> For example, support one operator resides in a tasks with parallelism 4,
> if 2 fo the subtasks are finished, now the state of the operator is
> composed
> of the state of the 2 remaining subtask instance, if we rescale to 5 after
> failover, the state of the 2 previous remaining subtasks would be
> re-distributed
> to the 5 new subtasks after failover.
>
> If before failover all the 4 subtasks are finished, the operator would be
> marked as finished, after failover the operator would be still marked as
> finished,
> and all the subtask instance of this operator would skip all the methods
> like open(), endOfInput(), close() and would be excluded when taking
> checkpoints
> after failover.
>
>
> > 3. Option 3 assumes that the state of a finished task is not used.
> That's true for operator state, but what about channel state (captured by
> unaligned checkpoint)?
> > I think it still has to be sent downstream which invalidates this Option.
>
> For unaligned checkpoint, if in one checkpoint a subtask is marked as
> finished, then its descandent tasks would wait all the records are received
> from the finished tasks before taking checkpoint, thus in this case we
> would not have result partition state, but only have channel state for the
> downstream tasks that are still running.
>
> 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" to trigger checkpoint via RPC. For task B, if
> it received checkpoint trigger, it would know that all its precedant tasks
> are finished, then it would wait till all the InputChannel received
> EndOfPartition from the network (namely inputChannel.onBuffer() 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 more clear~
>
> Best,
>  Yun
>
>
> ------------------Original Mail ------------------
> *Sender:*Khachatryan Roman <kh...@gmail.com>
> *Send Date:*Thu Jan 7 21:55:52 2021
> *Recipients:*Arvid Heise <ar...@ververica.com>
> *CC:*dev <de...@flink.apache.org>, user <us...@flink.apache.org>
> *Subject:*Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
>> 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 short time until
>> the job finishes completely.
>> And compared to other options, complexity seems much lower. Or are
>> differences in task completion times so huge and so common?
>>
>> 2. I think it would be helpful to describe how is rescaling handled in
>> Options 2 and 3 (or maybe it's not supported for jobs about to finish).
>>
>> 3. Option 3 assumes that the state of a finished task is not used. That's
>> true for operator state, but what about channel state (captured by
>> unaligned checkpoint)? I think it still has to be sent downstream which
>> invalidates this Option.
>>
>> Regards,
>> Roman
>>
>>
>> On Thu, Jan 7, 2021 at 1:21 PM Arvid Heise <ar...@ververica.com> wrote:
>>
>>> 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 interface would have a method like `isFinalized()` that the
>>>> framework can call after each checkpoint (and potentially at other
>>>> points)
>>>
>>>
>>> I think we are mixing two different things here that may require
>>> different solutions:
>>> 1. Tasks (=sink) that may need to do something with the final checkpoint.
>>> 2. Tasks that only finish after having finished operations that do not
>>> depend on data flow (async I/O, but I could also think of some timer
>>> actions in process functions).
>>>
>>> Your proposal would help most for the first case. The second case can
>>> solved entirely with current methods without being especially complicated:
>>> - EOP is only emitted once Async I/O is done with all background tasks
>>> - All timers are fired in a process function (I think we rather want to
>>> fire immediately on EOP but that's a different discussion)
>>> The advantage of this approach over your idea is that you don't need to
>>> wait for a checkpoint to complete to check for finalization.
>>>
>>> Now let's look at the first case. I see two alternatives:
>>> - The new sink interface implicitly incorporates this listener. Since I
>>> don't see a use case outside sinks, we could simply add this method to the
>>> new sink interface.
>>> - We implicitly assume that a sink is done after having a successful
>>> checkpoint at the end. Then we just need a tag interface
>>> `RequiresFinalization`. It also feels like we should add the property
>>> `final` to checkpoint options to help the sink detect that this is the last
>>> checkpoint to be taken. We could also try to always have the final
>>> checkpoint without tag interface on new sinks...
>>>
>>> On Thu, Jan 7, 2021 at 11:58 AM Aljoscha Krettek <al...@apache.org>
>>> wrote:
>>>
>>>> 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 least for Async I/O and
>>>> potentially for sinks.
>>>>
>>>> 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 interface would have a method like `isFinalized()` that the
>>>> framework can call after each checkpoint (and potentially at other
>>>> points)
>>>>
>>>> This way we would decouple that logic from things that don't actually
>>>> need it. What do you think?
>>>>
>>>> Best,
>>>> Aljoscha
>>>>
>>>
>>>
>>> --
>>>
>>> Arvid Heise | Senior Java Developer
>>>
>>> <https://www.ververica.com/>
>>>
>>> Follow us @VervericaData
>>>
>>> --
>>>
>>> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
>>> Conference
>>>
>>> Stream Processing | Event Driven | Real Time
>>>
>>> --
>>>
>>> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>>>
>>> --
>>> Ververica GmbH
>>> Registered at Amtsgericht Charlottenburg: HRB 158244 B
>>> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
>>> (Toni) Cheng
>>>
>>

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi Roman,

   Very thanks for the feedbacks! I'll try to answer the issues inline:

> 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 short time until the job finishes completely. 
> And compared to other options, complexity seems much lower. Or are differences in task completion times so huge and so common?

There might be mixed jobs with both bounded sources and unbounded sources, in this case, the resource for the finished 
part of the job would not be able to be released.

And the Option 1 also complicates the semantics of the EndOfPartition, since if we holding the tasks and we still need to
notify the following tasks about all records are sent, we would have to introduce some kind of pre-EndOfPartition messages, 
which is similar to the current EndOfPartition, but do not cause the channels to be released.

> 2. I think it would be helpful to describe how is rescaling handled in Options 2 and 3 (or maybe it's not supported for jobs about to finish).

For Option 2 and 3 we managed the states via the unit of operator, thus the process of rescaling would be the same with the normal checkpoint.
For example, support one operator resides in a tasks with parallelism 4, if 2 fo the subtasks are finished, now the state of the operator is composed 
of the state of the 2 remaining subtask instance, if we rescale to 5 after failover, the state of the 2 previous remaining subtasks would be re-distributed 
to the 5 new subtasks after failover. 

If before failover all the 4 subtasks are finished, the operator would be marked as finished, after failover the operator would be still marked as finished, 
and all the subtask instance of this operator would skip all the methods like open(), endOfInput(), close() and would be excluded when taking checkpoints
 after failover.


> 3. Option 3 assumes that the state of a finished task is not used. That's true for operator state, but what about channel state (captured by unaligned checkpoint)?
> I think it still has to be sent downstream which invalidates this Option.

For unaligned checkpoint, if in one checkpoint a subtask is marked as finished, then its descandent tasks would wait all the records are received
 from the finished tasks before taking checkpoint, thus in this case we would not have result partition state, but only have channel state for the 
downstream tasks that are still running.

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" to trigger checkpoint via RPC. For task B, if it received checkpoint trigger, it would know that all its precedant tasks
are finished, then it would wait till all the InputChannel received EndOfPartition from the network (namely inputChannel.onBuffer() 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 more clear~

Best,
 Yun



 ------------------Original Mail ------------------
Sender:Khachatryan Roman <kh...@gmail.com>
Send Date:Thu Jan 7 21:55:52 2021
Recipients:Arvid Heise <ar...@ververica.com>
CC:dev <de...@flink.apache.org>, user <us...@flink.apache.org>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

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 short time until the job finishes completely. 
And compared to other options, complexity seems much lower. Or are differences in task completion times so huge and so common?

2. I think it would be helpful to describe how is rescaling handled in Options 2 and 3 (or maybe it's not supported for jobs about to finish).

3. Option 3 assumes that the state of a finished task is not used. That's true for operator state, but what about channel state (captured by unaligned checkpoint)? I think it still has to be sent downstream which invalidates this Option.

Regards,
Roman

On Thu, Jan 7, 2021 at 1:21 PM Arvid Heise <ar...@ververica.com> wrote:

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 interface would have a method like `isFinalized()` that the 
framework can call after each checkpoint (and potentially at other 
points)

I think we are mixing two different things here that may require different solutions:
1. Tasks (=sink) that may need to do something with the final checkpoint.
2. Tasks that only finish after having finished operations that do not depend on data flow (async I/O, but I could also think of some timer actions in process functions).

Your proposal would help most for the first case. The second case can solved entirely with current methods without being especially complicated: 
- EOP is only emitted once Async I/O is done with all background tasks
- All timers are fired in a process function (I think we rather want to fire immediately on EOP but that's a different discussion)
The advantage of this approach over your idea is that you don't need to wait for a checkpoint to complete to check for finalization.

Now let's look at the first case. I see two alternatives:
- The new sink interface implicitly incorporates this listener. Since I don't see a use case outside sinks, we could simply add this method to the new sink interface.
- We implicitly assume that a sink is done after having a successful checkpoint at the end. Then we just need a tag interface `RequiresFinalization`. It also feels like we should add the property `final` to checkpoint options to help the sink detect that this is the last checkpoint to be taken. We could also try to always have the final checkpoint without tag interface on new sinks...

On Thu, Jan 7, 2021 at 11:58 AM Aljoscha Krettek <al...@apache.org> wrote:
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 least for Async I/O and 
potentially for sinks.

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 interface would have a method like `isFinalized()` that the 
framework can call after each checkpoint (and potentially at other 
points)

This way we would decouple that logic from things that don't actually 
need it. What do you think?

Best,
Aljoscha


-- 
Arvid Heise | Senior Java Developer

Follow us @VervericaData
--
Join Flink Forward - The Apache Flink Conference
Stream Processing | Event Driven | Real Time
--
Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
--Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji (Toni) Cheng 

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

Posted by Yun Gao <yu...@aliyun.com>.
Hi Roman,

   Very thanks for the feedbacks! I'll try to answer the issues inline:

> 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 short time until the job finishes completely. 
> And compared to other options, complexity seems much lower. Or are differences in task completion times so huge and so common?

There might be mixed jobs with both bounded sources and unbounded sources, in this case, the resource for the finished 
part of the job would not be able to be released.

And the Option 1 also complicates the semantics of the EndOfPartition, since if we holding the tasks and we still need to
notify the following tasks about all records are sent, we would have to introduce some kind of pre-EndOfPartition messages, 
which is similar to the current EndOfPartition, but do not cause the channels to be released.

> 2. I think it would be helpful to describe how is rescaling handled in Options 2 and 3 (or maybe it's not supported for jobs about to finish).

For Option 2 and 3 we managed the states via the unit of operator, thus the process of rescaling would be the same with the normal checkpoint.
For example, support one operator resides in a tasks with parallelism 4, if 2 fo the subtasks are finished, now the state of the operator is composed 
of the state of the 2 remaining subtask instance, if we rescale to 5 after failover, the state of the 2 previous remaining subtasks would be re-distributed 
to the 5 new subtasks after failover. 

If before failover all the 4 subtasks are finished, the operator would be marked as finished, after failover the operator would be still marked as finished, 
and all the subtask instance of this operator would skip all the methods like open(), endOfInput(), close() and would be excluded when taking checkpoints
 after failover.


> 3. Option 3 assumes that the state of a finished task is not used. That's true for operator state, but what about channel state (captured by unaligned checkpoint)?
> I think it still has to be sent downstream which invalidates this Option.

For unaligned checkpoint, if in one checkpoint a subtask is marked as finished, then its descandent tasks would wait all the records are received
 from the finished tasks before taking checkpoint, thus in this case we would not have result partition state, but only have channel state for the 
downstream tasks that are still running.

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" to trigger checkpoint via RPC. For task B, if it received checkpoint trigger, it would know that all its precedant tasks
are finished, then it would wait till all the InputChannel received EndOfPartition from the network (namely inputChannel.onBuffer() 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 more clear~

Best,
 Yun



 ------------------Original Mail ------------------
Sender:Khachatryan Roman <kh...@gmail.com>
Send Date:Thu Jan 7 21:55:52 2021
Recipients:Arvid Heise <ar...@ververica.com>
CC:dev <de...@flink.apache.org>, user <us...@flink.apache.org>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

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 short time until the job finishes completely. 
And compared to other options, complexity seems much lower. Or are differences in task completion times so huge and so common?

2. I think it would be helpful to describe how is rescaling handled in Options 2 and 3 (or maybe it's not supported for jobs about to finish).

3. Option 3 assumes that the state of a finished task is not used. That's true for operator state, but what about channel state (captured by unaligned checkpoint)? I think it still has to be sent downstream which invalidates this Option.

Regards,
Roman

On Thu, Jan 7, 2021 at 1:21 PM Arvid Heise <ar...@ververica.com> wrote:

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 interface would have a method like `isFinalized()` that the 
framework can call after each checkpoint (and potentially at other 
points)

I think we are mixing two different things here that may require different solutions:
1. Tasks (=sink) that may need to do something with the final checkpoint.
2. Tasks that only finish after having finished operations that do not depend on data flow (async I/O, but I could also think of some timer actions in process functions).

Your proposal would help most for the first case. The second case can solved entirely with current methods without being especially complicated: 
- EOP is only emitted once Async I/O is done with all background tasks
- All timers are fired in a process function (I think we rather want to fire immediately on EOP but that's a different discussion)
The advantage of this approach over your idea is that you don't need to wait for a checkpoint to complete to check for finalization.

Now let's look at the first case. I see two alternatives:
- The new sink interface implicitly incorporates this listener. Since I don't see a use case outside sinks, we could simply add this method to the new sink interface.
- We implicitly assume that a sink is done after having a successful checkpoint at the end. Then we just need a tag interface `RequiresFinalization`. It also feels like we should add the property `final` to checkpoint options to help the sink detect that this is the last checkpoint to be taken. We could also try to always have the final checkpoint without tag interface on new sinks...

On Thu, Jan 7, 2021 at 11:58 AM Aljoscha Krettek <al...@apache.org> wrote:
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 least for Async I/O and 
potentially for sinks.

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 interface would have a method like `isFinalized()` that the 
framework can call after each checkpoint (and potentially at other 
points)

This way we would decouple that logic from things that don't actually 
need it. What do you think?

Best,
Aljoscha


-- 
Arvid Heise | Senior Java Developer

Follow us @VervericaData
--
Join Flink Forward - The Apache Flink Conference
Stream Processing | Event Driven | Real Time
--
Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
--Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji (Toni) Cheng 

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

Posted by Khachatryan Roman <kh...@gmail.com>.
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 short time until
the job finishes completely.
And compared to other options, complexity seems much lower. Or are
differences in task completion times so huge and so common?

2. I think it would be helpful to describe how is rescaling handled in
Options 2 and 3 (or maybe it's not supported for jobs about to finish).

3. Option 3 assumes that the state of a finished task is not used. That's
true for operator state, but what about channel state (captured by
unaligned checkpoint)? I think it still has to be sent downstream which
invalidates this Option.

Regards,
Roman


On Thu, Jan 7, 2021 at 1:21 PM Arvid Heise <ar...@ververica.com> wrote:

> 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 interface would have a method like `isFinalized()` that the
>> framework can call after each checkpoint (and potentially at other
>> points)
>
>
> I think we are mixing two different things here that may require different
> solutions:
> 1. Tasks (=sink) that may need to do something with the final checkpoint.
> 2. Tasks that only finish after having finished operations that do not
> depend on data flow (async I/O, but I could also think of some timer
> actions in process functions).
>
> Your proposal would help most for the first case. The second case can
> solved entirely with current methods without being especially complicated:
> - EOP is only emitted once Async I/O is done with all background tasks
> - All timers are fired in a process function (I think we rather want to
> fire immediately on EOP but that's a different discussion)
> The advantage of this approach over your idea is that you don't need to
> wait for a checkpoint to complete to check for finalization.
>
> Now let's look at the first case. I see two alternatives:
> - The new sink interface implicitly incorporates this listener. Since I
> don't see a use case outside sinks, we could simply add this method to the
> new sink interface.
> - We implicitly assume that a sink is done after having a successful
> checkpoint at the end. Then we just need a tag interface
> `RequiresFinalization`. It also feels like we should add the property
> `final` to checkpoint options to help the sink detect that this is the last
> checkpoint to be taken. We could also try to always have the final
> checkpoint without tag interface on new sinks...
>
> On Thu, Jan 7, 2021 at 11:58 AM Aljoscha Krettek <al...@apache.org>
> wrote:
>
>> 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 least for Async I/O and
>> potentially for sinks.
>>
>> 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 interface would have a method like `isFinalized()` that the
>> framework can call after each checkpoint (and potentially at other
>> points)
>>
>> This way we would decouple that logic from things that don't actually
>> need it. What do you think?
>>
>> Best,
>> Aljoscha
>>
>
>
> --
>
> Arvid Heise | Senior Java Developer
>
> <https://www.ververica.com/>
>
> Follow us @VervericaData
>
> --
>
> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
> Conference
>
> Stream Processing | Event Driven | Real Time
>
> --
>
> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>
> --
> Ververica GmbH
> Registered at Amtsgericht Charlottenburg: HRB 158244 B
> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
> (Toni) Cheng
>

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

Posted by Khachatryan Roman <kh...@gmail.com>.
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 short time until
the job finishes completely.
And compared to other options, complexity seems much lower. Or are
differences in task completion times so huge and so common?

2. I think it would be helpful to describe how is rescaling handled in
Options 2 and 3 (or maybe it's not supported for jobs about to finish).

3. Option 3 assumes that the state of a finished task is not used. That's
true for operator state, but what about channel state (captured by
unaligned checkpoint)? I think it still has to be sent downstream which
invalidates this Option.

Regards,
Roman


On Thu, Jan 7, 2021 at 1:21 PM Arvid Heise <ar...@ververica.com> wrote:

> 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 interface would have a method like `isFinalized()` that the
>> framework can call after each checkpoint (and potentially at other
>> points)
>
>
> I think we are mixing two different things here that may require different
> solutions:
> 1. Tasks (=sink) that may need to do something with the final checkpoint.
> 2. Tasks that only finish after having finished operations that do not
> depend on data flow (async I/O, but I could also think of some timer
> actions in process functions).
>
> Your proposal would help most for the first case. The second case can
> solved entirely with current methods without being especially complicated:
> - EOP is only emitted once Async I/O is done with all background tasks
> - All timers are fired in a process function (I think we rather want to
> fire immediately on EOP but that's a different discussion)
> The advantage of this approach over your idea is that you don't need to
> wait for a checkpoint to complete to check for finalization.
>
> Now let's look at the first case. I see two alternatives:
> - The new sink interface implicitly incorporates this listener. Since I
> don't see a use case outside sinks, we could simply add this method to the
> new sink interface.
> - We implicitly assume that a sink is done after having a successful
> checkpoint at the end. Then we just need a tag interface
> `RequiresFinalization`. It also feels like we should add the property
> `final` to checkpoint options to help the sink detect that this is the last
> checkpoint to be taken. We could also try to always have the final
> checkpoint without tag interface on new sinks...
>
> On Thu, Jan 7, 2021 at 11:58 AM Aljoscha Krettek <al...@apache.org>
> wrote:
>
>> 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 least for Async I/O and
>> potentially for sinks.
>>
>> 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 interface would have a method like `isFinalized()` that the
>> framework can call after each checkpoint (and potentially at other
>> points)
>>
>> This way we would decouple that logic from things that don't actually
>> need it. What do you think?
>>
>> Best,
>> Aljoscha
>>
>
>
> --
>
> Arvid Heise | Senior Java Developer
>
> <https://www.ververica.com/>
>
> Follow us @VervericaData
>
> --
>
> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
> Conference
>
> Stream Processing | Event Driven | Real Time
>
> --
>
> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>
> --
> Ververica GmbH
> Registered at Amtsgericht Charlottenburg: HRB 158244 B
> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
> (Toni) Cheng
>

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

Posted by Arvid Heise <ar...@ververica.com>.
>
> 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 interface would have a method like `isFinalized()` that the
> framework can call after each checkpoint (and potentially at other
> points)


I think we are mixing two different things here that may require different
solutions:
1. Tasks (=sink) that may need to do something with the final checkpoint.
2. Tasks that only finish after having finished operations that do not
depend on data flow (async I/O, but I could also think of some timer
actions in process functions).

Your proposal would help most for the first case. The second case can
solved entirely with current methods without being especially complicated:
- EOP is only emitted once Async I/O is done with all background tasks
- All timers are fired in a process function (I think we rather want to
fire immediately on EOP but that's a different discussion)
The advantage of this approach over your idea is that you don't need to
wait for a checkpoint to complete to check for finalization.

Now let's look at the first case. I see two alternatives:
- The new sink interface implicitly incorporates this listener. Since I
don't see a use case outside sinks, we could simply add this method to the
new sink interface.
- We implicitly assume that a sink is done after having a successful
checkpoint at the end. Then we just need a tag interface
`RequiresFinalization`. It also feels like we should add the property
`final` to checkpoint options to help the sink detect that this is the last
checkpoint to be taken. We could also try to always have the final
checkpoint without tag interface on new sinks...

On Thu, Jan 7, 2021 at 11:58 AM Aljoscha Krettek <al...@apache.org>
wrote:

> 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 least for Async I/O and
> potentially for sinks.
>
> 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 interface would have a method like `isFinalized()` that the
> framework can call after each checkpoint (and potentially at other
> points)
>
> This way we would decouple that logic from things that don't actually
> need it. What do you think?
>
> Best,
> Aljoscha
>


-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

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

Posted by Arvid Heise <ar...@ververica.com>.
>
> 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 interface would have a method like `isFinalized()` that the
> framework can call after each checkpoint (and potentially at other
> points)


I think we are mixing two different things here that may require different
solutions:
1. Tasks (=sink) that may need to do something with the final checkpoint.
2. Tasks that only finish after having finished operations that do not
depend on data flow (async I/O, but I could also think of some timer
actions in process functions).

Your proposal would help most for the first case. The second case can
solved entirely with current methods without being especially complicated:
- EOP is only emitted once Async I/O is done with all background tasks
- All timers are fired in a process function (I think we rather want to
fire immediately on EOP but that's a different discussion)
The advantage of this approach over your idea is that you don't need to
wait for a checkpoint to complete to check for finalization.

Now let's look at the first case. I see two alternatives:
- The new sink interface implicitly incorporates this listener. Since I
don't see a use case outside sinks, we could simply add this method to the
new sink interface.
- We implicitly assume that a sink is done after having a successful
checkpoint at the end. Then we just need a tag interface
`RequiresFinalization`. It also feels like we should add the property
`final` to checkpoint options to help the sink detect that this is the last
checkpoint to be taken. We could also try to always have the final
checkpoint without tag interface on new sinks...

On Thu, Jan 7, 2021 at 11:58 AM Aljoscha Krettek <al...@apache.org>
wrote:

> 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 least for Async I/O and
> potentially for sinks.
>
> 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 interface would have a method like `isFinalized()` that the
> framework can call after each checkpoint (and potentially at other
> points)
>
> This way we would decouple that logic from things that don't actually
> need it. What do you think?
>
> Best,
> Aljoscha
>


-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

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

Posted by Aljoscha Krettek <al...@apache.org>.
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 least for Async I/O and 
potentially for sinks.

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 interface would have a method like `isFinalized()` that the 
framework can call after each checkpoint (and potentially at other 
points)

This way we would decouple that logic from things that don't actually 
need it. What do you think?

Best,
Aljoscha

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

Posted by Arvid Heise <ar...@ververica.com>.
Okay then at least you guys are in sync ;) (Although I'm also not too far
away)

I hope I'm not super derailing but could we reiterate why it's good to get
rid of finished tasks (note: I'm also mostly in favor of that):
1. We can free all acquired resources including buffer pools, state
backend(?), threads.
2. TM can forget about the subtask entirely.
3. We can subsequently downscale.
4. What more?

I'm assuming it's not needed to execute the application at all: The
application at one point had all subtasks running, so it's not a resource
issue per se (ignoring rescaling).

My idea is not to let the task live longer (except for final checkpoints
where we are all on the same page I guess). I'm just thinking out loud if
we can avoid 2. while still doing 1.+3.

So can TM retain some slim information about a finished task to still
process RPCs in a potentially different way?
Thus, without keeping the costly task thread and operator chains, could we
implement some RPC handler that knows this is a finished task and forward
the barrier to the next task/TM?
Can we store this slim information in a checkpoint as an operator subtask
state?
Could we transfer this slim information in case of (dynamic) downscaling?

If this somehow works, we would not need to change much in the checkpoint
coordinator. He would always inject into sources. We could also ignore the
race conditions as long as the TM lives. Checkpointing times are also not
worse as with the live task.
Clear downside (assuming feasibility) is that we have two code paths that
would deal with barriers. We would also need to keep more information in
the TM but again at some point the complete subtask fitted.

On Wed, Jan 6, 2021 at 4:39 PM Aljoscha Krettek <al...@apache.org> wrote:

> On 2021/01/06 16:05, Arvid Heise wrote:
> >thanks for the detailed example. It feels like Aljoscha and you are also
> >not fully aligned yet. For me, it sounded as if Aljoscha would like to
> >avoid sending RPC to non-source subtasks.
>
> No, I think we need the triggering of intermediate operators.
>
> I was just thinking out loud about the potential scenarios where
> intermediate operators will in fact stay online, and how common they
> are.
>
> Also, I sent an explanation that is similar to Yuns. It seems we always
> write out mails in parallel and then sent them before checking. :-) So
> you always get two explanations of roughly the same thing.
>
> Best,
> Aljoscha
>


-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

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

Posted by Aljoscha Krettek <al...@apache.org>.
On 2021/01/06 16:05, Arvid Heise wrote:
>thanks for the detailed example. It feels like Aljoscha and you are also
>not fully aligned yet. For me, it sounded as if Aljoscha would like to
>avoid sending RPC to non-source subtasks.

No, I think we need the triggering of intermediate operators.

I was just thinking out loud about the potential scenarios where 
intermediate operators will in fact stay online, and how common they 
are.

Also, I sent an explanation that is similar to Yuns. It seems we always 
write out mails in parallel and then sent them before checking. :-) So 
you always get two explanations of roughly the same thing.

Best,
Aljoscha

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

Posted by Aljoscha Krettek <al...@apache.org>.
On 2021/01/06 13:35, Arvid Heise wrote:
>I was actually not thinking about concurrent checkpoints (and actually want
>to get rid of them once UC is established, since they are addressing the
>same thing).

I would give a yuge +1 to that. I don't see why we would need concurrent 
checkpoints in most cases. (Any case even?)

>However, I have the impression that you think mostly in terms of tasks and
>I mostly think in terms of subtasks. I especially want to have proper
>support for bounded sources where one partition is much larger than the
>other partitions (might be in conjunction with unbounded sources such that
>checkpointing is plausible to begin with). Hence, most of the subtasks are
>finished with one struggler remaining. In this case, the barriers are
>inserted now only in the struggling source subtask and potentially in any
>running downstream subtask.
>As far as I have understood, this would require barriers to be inserted
>downstream leading to similar race conditions.

No, I'm also thinking in terms of subtasks when it comes to triggering.  
As long as a subtask has at least one upstream task we don't need to 
manually trigger that task. A task will know which of its inputs have 
finished, so it will take those out of the calculation that waits for 
barriers from all upstream tasks. In the case where only a single 
upstream source is remaining the barriers from that task will then 
trigger checkpointing at the downstream task.

>I'm also concerned about the notion of a final checkpoint. What happens
>when this final checkpoint times out (checkpoint timeout > async timeout)
>or fails for a different reason? I'm currently more inclined to just let
>checkpoints work until the whole graph is completed (and thought this was
>the initial goal of the whole FLIP to being with). However, that would
>require subtasks to stay alive until they receive checkpiontCompleted
>callback (which is currently also not guaranteed)...

The idea is that the final checkpoint is whatever checkpoint succeeds in 
the end. When a task (and I mostly mean subtask when I say task) knows 
that it is done it waits for the next successful checkpoint and then 
shuts down.

This is a basic question, though: should we simply keep all tasks 
(subtasks) around forever until the whole graph shuts down? Our answer 
for this was *no*, so far. We would like to allow tasks to shut down, 
such that the resources are freed at that point.

Best,
Aljoscha

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

Posted by Arvid Heise <ar...@ververica.com>.
Hi Yun,

thanks for the detailed example. It feels like Aljoscha and you are also
not fully aligned yet. For me, it sounded as if Aljoscha would like to
avoid sending RPC to non-source subtasks.

I think we are still on the same page that we would like to trigger
> checkpoint periodically until the whole job is finished.
> I think in generaly we do not must force the checkpoint aligned with
> subtask finished, namely for example one operator
> might have the lifecycle that "taking one checkpoint -> emit some records
> -> taking another checkpoint -> emit more records -> finish",
> and do not need to must have to wait for one more checkpoint before
> finished. The second checkpoint just happens to be the "final" checkpoint
> of this operator.
> The only exception is that for sink operator that must wait for one more
> checkpoint to commit the last piece of data before finished, this kind of
> operators
> would be dealt with separately to force them to wait for checkpont before
> finished
>

Yes that sounds good. I was concerned of any "holding back" of barriers in
async I/O. I'd just hold back the EOP until all async threads finished and
forward barriers in the normal way.

That would then also be my solution for sinks - hold back EOP (=finish)
until checkpoint is done. My concern here is still that we would need to
have a reliable mechanism to notify checkpoint completed. Maybe we can use
the GlobalCommitter?

In this case, if we do not deal with specially, based on the current
> implementation, the trigger RPC would just be ignored, and the checkpoint
> would finally
> failed due to timeout since no tasks would report its state. But we would
> be able to remedy this checkpont: since the Source subtask 2 and the Async
> I/O
> subtask would report FINISHED status to JobMaster after we tries to
> trigger the tasks, and before the task has reported its snapshot for this
> checkpont.
> The CheckpontCoordinator would listen to the notification, when it
> received the notification, it would iterates its pending checkpoints to see
> if it has trigger this task but received FINISHED before its snapshot. If
> so, it would recompute the subtasks to trigger, and re-trigger the
> following tasks.
> Of couse this is one possible implementation and we might have other
> solutions to this problem. Do you think the process would still have some
> problems ?
>

Here I'm just concerned that we would overload JM. Especially if it's
cascading: A is triggered in A->B->C but finishes, JM computes B and
resends RPC but at that time B is also finished. Hence I was thinking of
using TMs instead and only fall back to JM if TM has exited.

On Wed, Jan 6, 2021 at 3:29 PM Yun Gao <yu...@aliyun.com> wrote:

> Hi Arvid,
>
> Very thanks for the feedbacks! I'll try to answer the questions inline:
>
> > I'm also concerned about the notion of a final checkpoint. What happens
> > when this final checkpoint times out (checkpoint timeout > async timeout)
> > or fails for a different reason? I'm currently more inclined to just let
> > checkpoints work until the whole graph is completed (and thought this was
> > the initial goal of the whole FLIP to being with).
>
> I think we are still on the same page that we would like to trigger
> checkpoint periodically until the whole job is finished.
> I think in generaly we do not must force the checkpoint aligned with
> subtask finished, namely for example one operator
> might have the lifecycle that "taking one checkpoint -> emit some records
> -> taking another checkpoint -> emit more records -> finish",
> and do not need to must have to wait for one more checkpoint before
> finished. The second checkpoint just happens to be the "final" checkpoint
> of this operator.
> The only exception is that for sink operator that must wait for one more
> checkpoint to commit the last piece of data before finished, this kind of
> operators
> would be dealt with separately to force them to wait for checkpont before
> finished.
>
> >
> However, I have the impression that you think mostly in terms of tasks and
> > I mostly think in terms of subtasks. I especially want to have proper
> > support for bounded sources where one partition is much larger than the
> >
>  other partitions (might be in conjunction with unbounded sources such that
>
> > checkpointing is plausible to begin with). Hence, most of the subtasks are
> > finished with one struggler remaining. In this case, the barriers are
> > inserted now only in the struggling source subtask and potentially in any
> > running downstream subtask.
> > As far as I have understood, this would require barriers to be inserted
> > downstream leading to similar race conditions.
>
> I might not fully understand the issue, but I'd like to further detail
> the expected process here:
>
> Source (subtask 0) ---+
>                                      |
> Source (subtask 1) ---+--> Async I/O (subtask 0) -> Sink (subtask 0).
>                                     |
> Source (subtask 2) ---+
>
>
> The async I/O subtask would have three input channels.
>
> case 1) Support source subtask 0 and 1 are finished and the Async I/O
> would received EndOfPartition from the corresponding
> channels. Now we happen to trigger a checkpoint, we in the remaining
> execution graph, the subtask 2 is the "source" of the
> graph. Then we would trigger source subtask 2 to start the checkpoint,
> source subtask 2 takes snapshot and emit barriers to
> Async I/O sutask. Async I/O subtask would found that 2/3 of its input
> channels have received Eof and received barrier from
> the remaining channel, then it knows the barriers are aligned, then it
> takes the snapshot and emit the barrier to the sink subtasks.
>
> case 2) Suppose the job continue to run and now source subtask 2 is also
> finished and now we are going to take another checkpoint,
> then we found that in the remaining execution graph the new "source" now
> is the Async I/O subtask. Then we would trigger this
> Async I/O instead (this is different from the current implementation).
> The Async I/O received the trigger and take its snapshot and
> emit barrier to the following sink subtask. (Of couse here the Async I/O
> subtask should have some method to wait till it received EoF
> from all the input channels before taking snapshot to keep consistent, but
> I think we could ignore the detail implementations first).
>
> For the race condition, it might happen if
> a) in case 1, the CheckpontCoordinator trigger Source subtask 2, but
> source subtask 2 report finished before the trigger RPC gets into the
> resided TaskManager.
> b) in case 2, the CheckpointCoordinator trigger Async I/O, but Async I/O
> subtask report finished before the trigger RPC gets into the resided
> TaskManager.
>
> In this case, if we do not deal with specially, based on the current
> implementation, the trigger RPC would just be ignored, and the checkpoint
> would finally
> failed due to timeout since no tasks would report its state. But we would
> be able to remedy this checkpont: since the Source subtask 2 and the Async
> I/O
> subtask would report FINISHED status to JobMaster after we tries to
> trigger the tasks, and before the task has reported its snapshot for this
> checkpont.
> The CheckpontCoordinator would listen to the notification, when it
> received the notification, it would iterates its pending checkpoints to see
> if it has trigger this task but received FINISHED before its snapshot. If
> so, it would recompute the subtasks to trigger, and re-trigger the
> following tasks.
> Of couse this is one possible implementation and we might have other
> solutions to this problem. Do you think the process would still have some
> problems ?
>
> > However, that would
> > require subtasks to stay alive until they receive checkpiontCompleted
> > callback (which is currently also not guaranteed)
>
> With the above process, I think the task would do not need to wait for
> receiving the checkpontCompleted callback? If it finished, the above
> process
> would try to trigger its following tasks.
>
> Best,
> Yun
>
>
>
>
>
>
>
> ------------------Original Mail ------------------
> *Sender:*Arvid Heise <ar...@ververica.com>
> *Send Date:*Wed Jan 6 20:42:56 2021
> *Recipients:*Aljoscha Krettek <al...@apache.org>
> *CC:*dev <de...@flink.apache.org>
> *Subject:*Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
>>
>> I was actually not thinking about concurrent checkpoints (and actually want
>> to get rid of them once UC is established, since they are addressing the
>> same thing).
>>
>> But your explanation definitely helped me to better understand the race
>> condition.
>>
>> However, I have the impression that you think mostly in terms of tasks and
>> I mostly think in terms of subtasks. I especially want to have proper
>> support for bounded sources where one partition is much larger than the
>> other partitions (might be in conjunction with unbounded sources such that
>> checkpointing is plausible to begin with). Hence, most of the subtasks are
>> finished with one struggler remaining. In this case, the barriers are
>> inserted now only in the struggling source subtask and potentially in any
>> running downstream subtask.
>> As far as I have understood, this would require barriers to be inserted
>> downstream leading to similar race conditions.
>>
>> I'm also concerned about the notion of a final checkpoint. What happens
>> when this final checkpoint times out (checkpoint timeout > async timeout)
>> or fails for a different reason? I'm currently more inclined to just let
>> checkpoints work until the whole graph is completed (and thought this was
>> the initial goal of the whole FLIP to being with). However, that would
>> require subtasks to stay alive until they receive checkpiontCompleted
>> callback (which is currently also not guaranteed)...
>>
>> On Wed, Jan 6, 2021 at 12:17 PM Aljoscha Krettek <al...@apache.org>
>> wrote:
>>
>> > On 2021/01/06 11:30, Arvid Heise wrote:
>>
>> > >I'm assuming that this is the normal case. In a A->B graph, as soon as A
>> > >finishes, B still has a couple of input buffers to process. If you add
>> > >backpressure or longer pipelines into the mix, it's quite likely that a
>> > >checkpoint may occur with B being the head.
>> >
>> > Ahh, I think I know what you mean. This can happen when the checkpoint
>> > coordinator issues concurrent checkpoint without waiting for older ones
>> > to finish. My head is mostly operating under the premise that there is
>> > at most one concurrent checkpoint.
>> >
>> > In the current code base the race conditions that Yun and I are talking
>> > about cannot occur. Checkpoints can only be triggered at sources and
>> > they will then travel through the graph. Intermediate operators are
>> > never directly triggered from the JobManager/CheckpointCoordinator.
>> >
>> > When source start to shut down, the JM has to directly inject/trigger
>> > checkpoints at the now new "sources" of the graph, which have previously
>> > been intermediate operators.
>> >
>> > I want to repeat that I have a suspicion that maybe this is a degenerate
>> > case and we never want to allow operators to be doing checkpoints when
>> > they are not connected to at least one running source.  Which means that
>> > we have to find a solution for declined checkpoints, missing sources.
>> >
>> > I'll first show an example where I think we will never have intermediate
>> > operators running without the sources being running:
>> >
>> > Source -> Map -> Sink
>> >
>> > Here, when the Source does its final checkpoint and then shuts down,
>> > that same final checkpoint would travel downstream ahead of the EOF,
>> > which would in turn cause Map and Sink to also shut down. *We can't have
>> > the case that Map is still running when we want to take a checkpoint and
>> > Source is not running*.
>> >
>> > A similar case is this one:
>> >
>> > Source1 --+
>> >            |->Map -> Sink
>> > Source2 --+
>> >
>> > Here, if Source1 is finished but Source2 is not, Map is still connected
>> > to at least one upstream source that is still running. Again. Map would
>> > never be running and doing checkpoints if neither of Source1 or Source2
>> > are online.
>> >
>> > The cases I see where intermediate operators would keep running despite
>> > not being connected to any upstream operators are when we purposefully
>> > keep an operator online despite all inputs having seen EOF. One example
>> > is async I/O, another is what Yun mentioned where a sink might want to
>> > wait for another checkpoint to confirm some data. Example:
>> >
>> > Source -> Async I/O -> Sink
>> >
>> > Here, Async I/O will stay online as long as there are some scheduled
>> > requests outstanding, even when the Source has shut down. In those
>> > cases, the checkpoint coordinator would have to trigger new checkpoints
>> > at Async I/O and not Source, because it has become the new "head" of the
>> > graph.
>> >
>> > For Async I/O at least, we could say that the operator will wait for all
>> > outstanding requests to finish before it allows the final checkpoint and
>> > passes the barrier forward.
>> >
>> > Best,
>> > Aljoscha
>> >
>>
>>
>> --
>>
>> Arvid Heise | Senior Java Developer
>>
>> <https://www.ververica.com/>
>>
>> Follow us @VervericaData
>>
>> --
>>
>> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
>> Conference
>>
>> Stream Processing | Event Driven | Real Time
>>
>> --
>>
>> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>>
>> --
>> Ververica GmbH
>> Registered at Amtsgericht Charlottenburg: HRB 158244 B
>> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
>> (Toni) Cheng
>>
>

-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi Arvid,

Very thanks for the feedbacks! I'll try to answer the questions inline:

> I'm also concerned about the notion of a final checkpoint. What happens
> when this final checkpoint times out (checkpoint timeout > async timeout)
> or fails for a different reason? I'm currently more inclined to just let
> checkpoints work until the whole graph is completed (and thought this was
> the initial goal of the whole FLIP to being with).

I think we are still on the same page that we would like to trigger checkpoint periodically until the whole job is finished.
I think in generaly we do not must force the checkpoint aligned with subtask finished, namely for example one operator 
might have the lifecycle that "taking one checkpoint -> emit some records -> taking another checkpoint -> emit more records -> finish",
and do not need to must have to wait for one more checkpoint before finished. The second checkpoint just happens to be the "final" checkpoint of this operator. 
The only exception is that for sink operator that must wait for one more checkpoint to commit the last piece of data before finished, this kind of operators
would be dealt with separately to force them to wait for checkpont before finished.

> However, I have the impression that you think mostly in terms of tasks and
> I mostly think in terms of subtasks. I especially want to have proper
> support for bounded sources where one partition is much larger than the
>  other partitions (might be in conjunction with unbounded sources such that
> checkpointing is plausible to begin with). Hence, most of the subtasks are
> finished with one struggler remaining. In this case, the barriers are
> inserted now only in the struggling source subtask and potentially in any
> running downstream subtask.
> As far as I have understood, this would require barriers to be inserted
> downstream leading to similar race conditions.

I might not fully understand the issue, but I'd like to further detail
the expected process here: 

Source (subtask 0) ---+
                                     |
Source (subtask 1) ---+--> Async I/O (subtask 0) -> Sink (subtask 0).
                                    |
Source (subtask 2) ---+


The async I/O subtask would have three input channels. 

case 1) Support source subtask 0 and 1 are finished and the Async I/O would received EndOfPartition from the corresponding 
channels. Now we happen to trigger a checkpoint, we in the remaining execution graph, the subtask 2 is the "source" of the 
graph. Then we would trigger source subtask 2 to start the checkpoint, source subtask 2 takes snapshot and emit barriers to 
Async I/O sutask. Async I/O subtask would found that 2/3 of its input channels have received Eof and received barrier from 
the remaining channel, then it knows the barriers are aligned, then it takes the snapshot and emit the barrier to the sink subtasks.

case 2) Suppose the job continue to run and now source subtask 2 is also finished and now we are going to take another checkpoint, 
then we found that in the remaining execution graph the new "source" now is the Async I/O subtask. Then we would trigger this
Async I/O instead (this is different from the current implementation). The Async I/O received the trigger and take its snapshot and 
emit barrier to the following sink subtask. (Of couse here the Async I/O subtask should have some method to wait till it received EoF
from all the input channels before taking snapshot to keep consistent, but I think we could ignore the detail implementations first).

For the race condition, it might happen if 
a) in case 1, the CheckpontCoordinator trigger Source subtask 2, but source subtask 2 report finished before the trigger RPC gets into the resided TaskManager.
b) in case 2, the CheckpointCoordinator trigger Async I/O, but Async I/O subtask report finished before the trigger RPC gets into the resided TaskManager.

In this case, if we do not deal with specially, based on the current implementation, the trigger RPC would just be ignored, and the checkpoint would finally 
failed due to timeout since no tasks would report its state. But we would be able to remedy this checkpont: since the Source subtask 2 and the Async I/O
subtask would report FINISHED status to JobMaster after we tries to trigger the tasks, and before the task has reported its snapshot for this checkpont. 
The CheckpontCoordinator would listen to the notification, when it received the notification, it would iterates its pending checkpoints to see 
if it has trigger this task but received FINISHED before its snapshot. If so, it would recompute the subtasks to trigger, and re-trigger the following tasks. 
Of couse this is one possible implementation and we might have other solutions to this problem. Do you think the process would still have some problems ?

> However, that would
> require subtasks to stay alive until they receive checkpiontCompleted
> callback (which is currently also not guaranteed)

With the above process, I think the task would do not need to wait for receiving the checkpontCompleted callback? If it finished, the above process 
would try to trigger its following tasks. 

Best,
Yun








 ------------------Original Mail ------------------
Sender:Arvid Heise <ar...@ververica.com>
Send Date:Wed Jan 6 20:42:56 2021
Recipients:Aljoscha Krettek <al...@apache.org>
CC:dev <de...@flink.apache.org>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
I was actually not thinking about concurrent checkpoints (and actually want
to get rid of them once UC is established, since they are addressing the
same thing).

But your explanation definitely helped me to better understand the race
condition.

However, I have the impression that you think mostly in terms of tasks and
I mostly think in terms of subtasks. I especially want to have proper
support for bounded sources where one partition is much larger than the
other partitions (might be in conjunction with unbounded sources such that
checkpointing is plausible to begin with). Hence, most of the subtasks are
finished with one struggler remaining. In this case, the barriers are
inserted now only in the struggling source subtask and potentially in any
running downstream subtask.
As far as I have understood, this would require barriers to be inserted
downstream leading to similar race conditions.

I'm also concerned about the notion of a final checkpoint. What happens
when this final checkpoint times out (checkpoint timeout > async timeout)
or fails for a different reason? I'm currently more inclined to just let
checkpoints work until the whole graph is completed (and thought this was
the initial goal of the whole FLIP to being with). However, that would
require subtasks to stay alive until they receive checkpiontCompleted
callback (which is currently also not guaranteed)...

On Wed, Jan 6, 2021 at 12:17 PM Aljoscha Krettek <al...@apache.org>
wrote:

> On 2021/01/06 11:30, Arvid Heise wrote:
> >I'm assuming that this is the normal case. In a A->B graph, as soon as A
> >finishes, B still has a couple of input buffers to process. If you add
> >backpressure or longer pipelines into the mix, it's quite likely that a
> >checkpoint may occur with B being the head.
>
> Ahh, I think I know what you mean. This can happen when the checkpoint
> coordinator issues concurrent checkpoint without waiting for older ones
> to finish. My head is mostly operating under the premise that there is
> at most one concurrent checkpoint.
>
> In the current code base the race conditions that Yun and I are talking
> about cannot occur. Checkpoints can only be triggered at sources and
> they will then travel through the graph. Intermediate operators are
> never directly triggered from the JobManager/CheckpointCoordinator.
>
> When source start to shut down, the JM has to directly inject/trigger
> checkpoints at the now new "sources" of the graph, which have previously
> been intermediate operators.
>
> I want to repeat that I have a suspicion that maybe this is a degenerate
> case and we never want to allow operators to be doing checkpoints when
> they are not connected to at least one running source.  Which means that
> we have to find a solution for declined checkpoints, missing sources.
>
> I'll first show an example where I think we will never have intermediate
> operators running without the sources being running:
>
> Source -> Map -> Sink
>
> Here, when the Source does its final checkpoint and then shuts down,
> that same final checkpoint would travel downstream ahead of the EOF,
> which would in turn cause Map and Sink to also shut down. *We can't have
> the case that Map is still running when we want to take a checkpoint and
> Source is not running*.
>
> A similar case is this one:
>
> Source1 --+
>            |->Map -> Sink
> Source2 --+
>
> Here, if Source1 is finished but Source2 is not, Map is still connected
> to at least one upstream source that is still running. Again. Map would
> never be running and doing checkpoints if neither of Source1 or Source2
> are online.
>
> The cases I see where intermediate operators would keep running despite
> not being connected to any upstream operators are when we purposefully
> keep an operator online despite all inputs having seen EOF. One example
> is async I/O, another is what Yun mentioned where a sink might want to
> wait for another checkpoint to confirm some data. Example:
>
> Source -> Async I/O -> Sink
>
> Here, Async I/O will stay online as long as there are some scheduled
> requests outstanding, even when the Source has shut down. In those
> cases, the checkpoint coordinator would have to trigger new checkpoints
> at Async I/O and not Source, because it has become the new "head" of the
> graph.
>
> For Async I/O at least, we could say that the operator will wait for all
> outstanding requests to finish before it allows the final checkpoint and
> passes the barrier forward.
>
> Best,
> Aljoscha
>


-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

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

Posted by Arvid Heise <ar...@ververica.com>.
I was actually not thinking about concurrent checkpoints (and actually want
to get rid of them once UC is established, since they are addressing the
same thing).

But your explanation definitely helped me to better understand the race
condition.

However, I have the impression that you think mostly in terms of tasks and
I mostly think in terms of subtasks. I especially want to have proper
support for bounded sources where one partition is much larger than the
other partitions (might be in conjunction with unbounded sources such that
checkpointing is plausible to begin with). Hence, most of the subtasks are
finished with one struggler remaining. In this case, the barriers are
inserted now only in the struggling source subtask and potentially in any
running downstream subtask.
As far as I have understood, this would require barriers to be inserted
downstream leading to similar race conditions.

I'm also concerned about the notion of a final checkpoint. What happens
when this final checkpoint times out (checkpoint timeout > async timeout)
or fails for a different reason? I'm currently more inclined to just let
checkpoints work until the whole graph is completed (and thought this was
the initial goal of the whole FLIP to being with). However, that would
require subtasks to stay alive until they receive checkpiontCompleted
callback (which is currently also not guaranteed)...

On Wed, Jan 6, 2021 at 12:17 PM Aljoscha Krettek <al...@apache.org>
wrote:

> On 2021/01/06 11:30, Arvid Heise wrote:
> >I'm assuming that this is the normal case. In a A->B graph, as soon as A
> >finishes, B still has a couple of input buffers to process. If you add
> >backpressure or longer pipelines into the mix, it's quite likely that a
> >checkpoint may occur with B being the head.
>
> Ahh, I think I know what you mean. This can happen when the checkpoint
> coordinator issues concurrent checkpoint without waiting for older ones
> to finish. My head is mostly operating under the premise that there is
> at most one concurrent checkpoint.
>
> In the current code base the race conditions that Yun and I are talking
> about cannot occur. Checkpoints can only be triggered at sources and
> they will then travel through the graph. Intermediate operators are
> never directly triggered from the JobManager/CheckpointCoordinator.
>
> When source start to shut down, the JM has to directly inject/trigger
> checkpoints at the now new "sources" of the graph, which have previously
> been intermediate operators.
>
> I want to repeat that I have a suspicion that maybe this is a degenerate
> case and we never want to allow operators to be doing checkpoints when
> they are not connected to at least one running source.  Which means that
> we have to find a solution for declined checkpoints, missing sources.
>
> I'll first show an example where I think we will never have intermediate
> operators running without the sources being running:
>
> Source -> Map -> Sink
>
> Here, when the Source does its final checkpoint and then shuts down,
> that same final checkpoint would travel downstream ahead of the EOF,
> which would in turn cause Map and Sink to also shut down. *We can't have
> the case that Map is still running when we want to take a checkpoint and
> Source is not running*.
>
> A similar case is this one:
>
> Source1 --+
>            |->Map -> Sink
> Source2 --+
>
> Here, if Source1 is finished but Source2 is not, Map is still connected
> to at least one upstream source that is still running. Again. Map would
> never be running and doing checkpoints if neither of Source1 or Source2
> are online.
>
> The cases I see where intermediate operators would keep running despite
> not being connected to any upstream operators are when we purposefully
> keep an operator online despite all inputs having seen EOF. One example
> is async I/O, another is what Yun mentioned where a sink might want to
> wait for another checkpoint to confirm some data. Example:
>
> Source -> Async I/O -> Sink
>
> Here, Async I/O will stay online as long as there are some scheduled
> requests outstanding, even when the Source has shut down. In those
> cases, the checkpoint coordinator would have to trigger new checkpoints
> at Async I/O and not Source, because it has become the new "head" of the
> graph.
>
> For Async I/O at least, we could say that the operator will wait for all
> outstanding requests to finish before it allows the final checkpoint and
> passes the barrier forward.
>
> Best,
> Aljoscha
>


-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

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

Posted by Aljoscha Krettek <al...@apache.org>.
On 2021/01/06 11:30, Arvid Heise wrote:
>I'm assuming that this is the normal case. In a A->B graph, as soon as A
>finishes, B still has a couple of input buffers to process. If you add
>backpressure or longer pipelines into the mix, it's quite likely that a
>checkpoint may occur with B being the head.

Ahh, I think I know what you mean. This can happen when the checkpoint 
coordinator issues concurrent checkpoint without waiting for older ones 
to finish. My head is mostly operating under the premise that there is 
at most one concurrent checkpoint.

In the current code base the race conditions that Yun and I are talking 
about cannot occur. Checkpoints can only be triggered at sources and 
they will then travel through the graph. Intermediate operators are 
never directly triggered from the JobManager/CheckpointCoordinator.

When source start to shut down, the JM has to directly inject/trigger 
checkpoints at the now new "sources" of the graph, which have previously 
been intermediate operators.

I want to repeat that I have a suspicion that maybe this is a degenerate 
case and we never want to allow operators to be doing checkpoints when 
they are not connected to at least one running source.  Which means that 
we have to find a solution for declined checkpoints, missing sources.

I'll first show an example where I think we will never have intermediate 
operators running without the sources being running:

Source -> Map -> Sink

Here, when the Source does its final checkpoint and then shuts down, 
that same final checkpoint would travel downstream ahead of the EOF, 
which would in turn cause Map and Sink to also shut down. *We can't have 
the case that Map is still running when we want to take a checkpoint and 
Source is not running*.

A similar case is this one:

Source1 --+
           |->Map -> Sink
Source2 --+

Here, if Source1 is finished but Source2 is not, Map is still connected 
to at least one upstream source that is still running. Again. Map would 
never be running and doing checkpoints if neither of Source1 or Source2 
are online.

The cases I see where intermediate operators would keep running despite 
not being connected to any upstream operators are when we purposefully 
keep an operator online despite all inputs having seen EOF. One example 
is async I/O, another is what Yun mentioned where a sink might want to 
wait for another checkpoint to confirm some data. Example:

Source -> Async I/O -> Sink

Here, Async I/O will stay online as long as there are some scheduled 
requests outstanding, even when the Source has shut down. In those 
cases, the checkpoint coordinator would have to trigger new checkpoints 
at Async I/O and not Source, because it has become the new "head" of the 
graph.

For Async I/O at least, we could say that the operator will wait for all 
outstanding requests to finish before it allows the final checkpoint and 
passes the barrier forward.

Best,
Aljoscha

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

Posted by Arvid Heise <ar...@ververica.com>.
>
> I was referring to the case where intermediate operators don't have any
> active upstream (input) operators. In that case, they basically become
> the "source" of that part of the graph. In your example, M1 is still
> connected to a "real" source.


I'm assuming that this is the normal case. In a A->B graph, as soon as A
finishes, B still has a couple of input buffers to process. If you add
backpressure or longer pipelines into the mix, it's quite likely that a
checkpoint may occur with B being the head.

    For faked finished tasks, I have some concerns that if the faked
> finished tasks reside in the JM side, there should still be the race
> condition between triggering
>     and tasks get finished, and if the faked finished tasks reside in the
> TM side, we would have to keep consider these tasks in scheduler when
> failover happens.
>     Besides, we would also need to keep the channels between the faked
> finished tasks and normal tasks to pass the checkpoint barriers, this would
> have some conflicts with
>     the current tasks' lifecycle since we still need to keep channels open
> and send messages after EndOfPartitions are sent. If we have mixed jobs
> with both bounded and
>      unbounded sources, the left network channels would not have a chance
> to get closed.
>

These are all valid concerns but I fear that if we don't find a solution to
them, we will not have a reliable system (cancelling checkpoints when
encountering this race condition with a higher DOP).

Let me clarify that faked finished tasks should reside on the TM that they
previously lived. Only through some kind of job stealing that is necessary
for dynamic rescaling they may end up in JM (that's all far down the road).

I have not thought about channels, but I think you are right that channels
should be strictly bound to the life-cycle of a subtask. The question is if
fake finished tasks do need to use channels at all. We could also relay RPC
calls from TM to TM.

For all practical purposes on checkpoint barrier alignment, EndOfPartitions
should make the channel being excluded from alignment (the respective
channel has implicitly received all future barriers).

On Wed, Jan 6, 2021 at 10:46 AM Aljoscha Krettek <al...@apache.org>
wrote:

> On 2021/01/05 17:27, Arvid Heise wrote:
> >For your question: will there ever be intermediate operators that should
> be
> >running that are not connected to at least once source?
> >I think there are plenty of examples if you go beyond chained operators
> and
> >fully connected exchanges. Think of any fan-in, let's assume you have
> >source S1...S4, with S1+S2->M1, and S3+S4->M2. If S1 is finished, S2 and
> M1
> >is still running. Or I didn't get your question ;).
>
> I was referring to the case where intermediate operators don't have any
> active upstream (input) operators. In that case, they basically become
> the "source" of that part of the graph. In your example, M1 is still
> connected to a "real" source.
>
> Best,
> Aljoscha
>


-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

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

Posted by Aljoscha Krettek <al...@apache.org>.
On 2021/01/05 17:27, Arvid Heise wrote:
>For your question: will there ever be intermediate operators that should be
>running that are not connected to at least once source?
>I think there are plenty of examples if you go beyond chained operators and
>fully connected exchanges. Think of any fan-in, let's assume you have
>source S1...S4, with S1+S2->M1, and S3+S4->M2. If S1 is finished, S2 and M1
>is still running. Or I didn't get your question ;).

I was referring to the case where intermediate operators don't have any 
active upstream (input) operators. In that case, they basically become 
the "source" of that part of the graph. In your example, M1 is still 
connected to a "real" source.

Best,
Aljoscha

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
     Hi Arvid,

          Very thanks for the feedbacks!

         > For 2) the race condition, I was more thinking of still injecting the
         > barrier at the source in all cases, but having some kind of short-cut to
         > immediately execute the RPC inside the respective taskmanager. However,
         > that may prove hard in case of dynamic scale-ins. Nevertheless, because of
         > this race condition, we should still take some time to think about it as it
         > effectively means we need to support handling a barrier in a finished task
         > anyways. Maybe a finished task is still assigned to a TM with JM as a
         > fallback?
    For faked finished tasks, I have some concerns that if the faked finished tasks reside in the JM side, there should still be the race condition between triggering 
and tasks get finished, and if the faked finished tasks reside in the TM side, we would have to keep consider these tasks in scheduler when failover happens. 
    Besides, we would also need to keep the channels between the faked finished tasks and normal tasks to pass the checkpoint barriers, this would have some conflicts with 
    the current tasks' lifecycle since we still need to keep channels open and send messages after EndOfPartitions are sent. If we have mixed jobs with both bounded and
 unbounded sources, the left network channels would not have a chance to get closed.

   Best,
    Yun


------------------------------------------------------------------
From:Arvid Heise <ar...@ververica.com>
Send Time:2021 Jan. 6 (Wed.) 00:28
To:Yun Gao <yu...@aliyun.com>
Cc:Aljoscha Krettek <al...@apache.org>; dev <de...@flink.apache.org>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

For 2) the race condition, I was more thinking of still injecting the
barrier at the source in all cases, but having some kind of short-cut to
immediately execute the RPC inside the respective taskmanager. However,
that may prove hard in case of dynamic scale-ins. Nevertheless, because of
this race condition, we should still take some time to think about it as it
effectively means we need to support handling a barrier in a finished task
anyways. Maybe a finished task is still assigned to a TM with JM as a
fallback?

For your question: will there ever be intermediate operators that should be
running that are not connected to at least once source?
I think there are plenty of examples if you go beyond chained operators and
fully connected exchanges. Think of any fan-in, let's assume you have
source S1...S4, with S1+S2->M1, and S3+S4->M2. If S1 is finished, S2 and M1
is still running. Or I didn't get your question ;).

On Tue, Jan 5, 2021 at 5:00 PM Yun Gao <yu...@aliyun.com> wrote:

>      Hi Aljoscha,
>
>          Very thanks for the feedbacks!
>
>          For the second issue, I'm indeed thinking the race condition
> between deciding to trigger and operator get finished. And for this point,
>
>  >
> One thought here is this: will there ever be intermediate operators that
> > should be running that are not connected to at least once source? The
> > only case I can think of right now is async I/O. Or are there others? If
> > we think that there will never be intermediate operators that are not
> > connected to at least once source we might come up with a simpler
> > solution.
>
>      I think there are still cases that the intermediate operators runs
> with all its sources have finished, for example, source -> sink writer ->
> sink committer -> sink global committer,  since sink committer need to wait
> for one more checkpoint between endOfInput and close,
> it would continue to run after the source and sink writer are finished,
> until we could finish one checkpoint. And since the four operators could
> also be chained in one task, we may also need to consider the case that
> part of operators are finished when taking snapshot in
> of the tasks.
>
>    Best,
>     Yun
>
>
> ------------------------------------------------------------------
> From:Aljoscha Krettek <al...@apache.org>
> Send Time:2021 Jan. 5 (Tue.) 22:34
> To:dev <de...@flink.apache.org>
> Cc:Yun Gao <yu...@aliyun.com>
> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
> On 2021/01/05 10:16, Arvid Heise wrote:
> >1. I'd think that this is an orthogonal issue, which I'd solve separately.
> >My gut feeling says that this is something we should only address for new
> >sinks where we decouple the semantics of commits and checkpoints
> >anyways. @Aljoscha
> >Krettek <al...@apache.org> any idea on this one?
>
> I also think it's somewhat orthogonal, let's see where we land here once
> the other issues are hammered out.
>
> >2. I'm not sure I get it completely. Let's assume we have a source
>
> >partition that is finished before the first checkpoint. Then, we would need
> >to store the finished state of the subtask somehow. So I'm assuming, we
> >still need to trigger some checkpointing code on finished subtasks.
>
> What he's talking about here is the race condition between a) checkpoint
> coordinator decides to do a checkpoint and b) a source operator shuts
> down.
>
> Normally, the checkpoint coordinator only needs to trigger sources, and
> not intermediate operators. When we allow sources to shut down,
> intermediate operators now can become the "head" of a pipeline and
> become the things that need to be triggered.
>
> One thought here is this: will there ever be intermediate operators that
> should be running that are not connected to at least once source? The
> only case I can think of right now is async I/O. Or are there others? If
> we think that there will never be intermediate operators that are not
> connected to at least once source we might come up with a simpler
> solution.
>
> >3. Do we really want to store the finished flag in OperatorState? I was
> >assuming we want to have it more fine-grained on OperatorSubtaskState.
> >Maybe we can store the flag inside managed or raw state without changing
> >the format?
>
> I think we cannot store it in `OperatorSubtaskState` because of how
> operator state (the actual `ListState` that operators use) is reshuffled
> on restore to all operators. So normally it doesn't make sense to say
> that one of the subtasks is done when operator state is involved. Only
> when all subtasks are done can we record this operator as done, I think.
>
> Best,
> Aljoscha
>
>
>

-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng


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

Posted by Arvid Heise <ar...@ververica.com>.
For 2) the race condition, I was more thinking of still injecting the
barrier at the source in all cases, but having some kind of short-cut to
immediately execute the RPC inside the respective taskmanager. However,
that may prove hard in case of dynamic scale-ins. Nevertheless, because of
this race condition, we should still take some time to think about it as it
effectively means we need to support handling a barrier in a finished task
anyways. Maybe a finished task is still assigned to a TM with JM as a
fallback?

For your question: will there ever be intermediate operators that should be
running that are not connected to at least once source?
I think there are plenty of examples if you go beyond chained operators and
fully connected exchanges. Think of any fan-in, let's assume you have
source S1...S4, with S1+S2->M1, and S3+S4->M2. If S1 is finished, S2 and M1
is still running. Or I didn't get your question ;).

On Tue, Jan 5, 2021 at 5:00 PM Yun Gao <yu...@aliyun.com> wrote:

>      Hi Aljoscha,
>
>          Very thanks for the feedbacks!
>
>          For the second issue, I'm indeed thinking the race condition
> between deciding to trigger and operator get finished. And for this point,
>
>  >
> One thought here is this: will there ever be intermediate operators that
> > should be running that are not connected to at least once source? The
> > only case I can think of right now is async I/O. Or are there others? If
> > we think that there will never be intermediate operators that are not
> > connected to at least once source we might come up with a simpler
> > solution.
>
>      I think there are still cases that the intermediate operators runs
> with all its sources have finished, for example, source -> sink writer ->
> sink committer -> sink global committer,  since sink committer need to wait
> for one more checkpoint between endOfInput and close,
> it would continue to run after the source and sink writer are finished,
> until we could finish one checkpoint. And since the four operators could
> also be chained in one task, we may also need to consider the case that
> part of operators are finished when taking snapshot in
> of the tasks.
>
>    Best,
>     Yun
>
>
> ------------------------------------------------------------------
> From:Aljoscha Krettek <al...@apache.org>
> Send Time:2021 Jan. 5 (Tue.) 22:34
> To:dev <de...@flink.apache.org>
> Cc:Yun Gao <yu...@aliyun.com>
> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
> On 2021/01/05 10:16, Arvid Heise wrote:
> >1. I'd think that this is an orthogonal issue, which I'd solve separately.
> >My gut feeling says that this is something we should only address for new
> >sinks where we decouple the semantics of commits and checkpoints
> >anyways. @Aljoscha
> >Krettek <al...@apache.org> any idea on this one?
>
> I also think it's somewhat orthogonal, let's see where we land here once
> the other issues are hammered out.
>
> >2. I'm not sure I get it completely. Let's assume we have a source
>
> >partition that is finished before the first checkpoint. Then, we would need
> >to store the finished state of the subtask somehow. So I'm assuming, we
> >still need to trigger some checkpointing code on finished subtasks.
>
> What he's talking about here is the race condition between a) checkpoint
> coordinator decides to do a checkpoint and b) a source operator shuts
> down.
>
> Normally, the checkpoint coordinator only needs to trigger sources, and
> not intermediate operators. When we allow sources to shut down,
> intermediate operators now can become the "head" of a pipeline and
> become the things that need to be triggered.
>
> One thought here is this: will there ever be intermediate operators that
> should be running that are not connected to at least once source? The
> only case I can think of right now is async I/O. Or are there others? If
> we think that there will never be intermediate operators that are not
> connected to at least once source we might come up with a simpler
> solution.
>
> >3. Do we really want to store the finished flag in OperatorState? I was
> >assuming we want to have it more fine-grained on OperatorSubtaskState.
> >Maybe we can store the flag inside managed or raw state without changing
> >the format?
>
> I think we cannot store it in `OperatorSubtaskState` because of how
> operator state (the actual `ListState` that operators use) is reshuffled
> on restore to all operators. So normally it doesn't make sense to say
> that one of the subtasks is done when operator state is involved. Only
> when all subtasks are done can we record this operator as done, I think.
>
> Best,
> Aljoscha
>
>
>

-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
     Hi Aljoscha, 

         Very thanks for the feedbacks!

         For the second issue, I'm indeed thinking the race condition between deciding to trigger and operator get finished. And for this point,
 > One thought here is this: will there ever be intermediate operators that 
> should be running that are not connected to at least once source? The 
> only case I can think of right now is async I/O. Or are there others? If 
> we think that there will never be intermediate operators that are not 
> connected to at least once source we might come up with a simpler 
> solution.
     I think there are still cases that the intermediate operators runs with all its sources have finished, for example, source -> sink writer -> sink committer -> sink global committer,  since sink committer need to wait for one more checkpoint between endOfInput and close, 
it would continue to run after the source and sink writer are finished, until we could finish one checkpoint. And since the four operators could also be chained in one task, we may also need to consider the case that part of operators are finished when taking snapshot in
of the tasks.

   Best,
    Yun



------------------------------------------------------------------
From:Aljoscha Krettek <al...@apache.org>
Send Time:2021 Jan. 5 (Tue.) 22:34
To:dev <de...@flink.apache.org>
Cc:Yun Gao <yu...@aliyun.com>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

On 2021/01/05 10:16, Arvid Heise wrote:
>1. I'd think that this is an orthogonal issue, which I'd solve separately.
>My gut feeling says that this is something we should only address for new
>sinks where we decouple the semantics of commits and checkpoints
>anyways. @Aljoscha
>Krettek <al...@apache.org> any idea on this one?

I also think it's somewhat orthogonal, let's see where we land here once 
the other issues are hammered out.

>2. I'm not sure I get it completely. Let's assume we have a source
>partition that is finished before the first checkpoint. Then, we would need
>to store the finished state of the subtask somehow. So I'm assuming, we
>still need to trigger some checkpointing code on finished subtasks.

What he's talking about here is the race condition between a) checkpoint 
coordinator decides to do a checkpoint and b) a source operator shuts 
down.

Normally, the checkpoint coordinator only needs to trigger sources, and 
not intermediate operators. When we allow sources to shut down, 
intermediate operators now can become the "head" of a pipeline and 
become the things that need to be triggered.

One thought here is this: will there ever be intermediate operators that 
should be running that are not connected to at least once source? The 
only case I can think of right now is async I/O. Or are there others? If 
we think that there will never be intermediate operators that are not 
connected to at least once source we might come up with a simpler 
solution.

>3. Do we really want to store the finished flag in OperatorState? I was
>assuming we want to have it more fine-grained on OperatorSubtaskState.
>Maybe we can store the flag inside managed or raw state without changing
>the format?

I think we cannot store it in `OperatorSubtaskState` because of how 
operator state (the actual `ListState` that operators use) is reshuffled 
on restore to all operators. So normally it doesn't make sense to say 
that one of the subtasks is done when operator state is involved. Only 
when all subtasks are done can we record this operator as done, I think.

Best,
Aljoscha


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

Posted by Aljoscha Krettek <al...@apache.org>.
On 2021/01/05 10:16, Arvid Heise wrote:
>1. I'd think that this is an orthogonal issue, which I'd solve separately.
>My gut feeling says that this is something we should only address for new
>sinks where we decouple the semantics of commits and checkpoints
>anyways. @Aljoscha
>Krettek <al...@apache.org> any idea on this one?

I also think it's somewhat orthogonal, let's see where we land here once 
the other issues are hammered out.

>2. I'm not sure I get it completely. Let's assume we have a source
>partition that is finished before the first checkpoint. Then, we would need
>to store the finished state of the subtask somehow. So I'm assuming, we
>still need to trigger some checkpointing code on finished subtasks.

What he's talking about here is the race condition between a) checkpoint 
coordinator decides to do a checkpoint and b) a source operator shuts 
down.

Normally, the checkpoint coordinator only needs to trigger sources, and 
not intermediate operators. When we allow sources to shut down, 
intermediate operators now can become the "head" of a pipeline and 
become the things that need to be triggered.

One thought here is this: will there ever be intermediate operators that 
should be running that are not connected to at least once source? The 
only case I can think of right now is async I/O. Or are there others? If 
we think that there will never be intermediate operators that are not 
connected to at least once source we might come up with a simpler 
solution.

>3. Do we really want to store the finished flag in OperatorState? I was
>assuming we want to have it more fine-grained on OperatorSubtaskState.
>Maybe we can store the flag inside managed or raw state without changing
>the format?

I think we cannot store it in `OperatorSubtaskState` because of how 
operator state (the actual `ListState` that operators use) is reshuffled 
on restore to all operators. So normally it doesn't make sense to say 
that one of the subtasks is done when operator state is involved. Only 
when all subtasks are done can we record this operator as done, I think.

Best,
Aljoscha

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
     Hi Avrid, 

         Very thanks for the feedbacks!

         For the second issue, sorry I think I might not make it very clear, I'm initially thinking the case that for example for a job with graph A -> B -> C, when we compute which tasks to trigger, A is still running, so we trigger A to start the checkpoint. However, before the triggering message reached A, A gets finished and the trigger message failed due to not found the task. In this case if we do not handle it, the checkpoint would failed due to timeout. However, by default failed checkpoint would cause job failure and we would also need to wait for a checkpoint interval for the next checkpoint. One solution would be check all the pending checkpoints to trigger B instead when JM is notified that A is finished.

       For the third issue, it should work if we store a special value for some filed in OperatorState or OperatorSubtaskState, for example, we might store a special subtaskState map inside the OperatorState to mark it is finished since the finished operator should always have an empty state. Very thanks for the advices! I'll try with this method. 

Best,
 Yun



------------------------------------------------------------------
From:Arvid Heise <ar...@ververica.com>
Send Time:2021 Jan. 5 (Tue.) 17:16
To:Yun Gao <yu...@aliyun.com>
Cc:Aljoscha Krettek <al...@apache.org>; dev <de...@flink.apache.org>; user <us...@flink.apache.org>
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. My gut feeling says that this is something we should only address for new sinks where we decouple the semantics of commits and checkpoints anyways. @Aljoscha Krettek any idea on this one?

2. I'm not sure I get it completely. Let's assume we have a source partition that is finished before the first checkpoint. Then, we would need to store the finished state of the subtask somehow. So I'm assuming, we still need to trigger some checkpointing code on finished subtasks.

3. Do we really want to store the finished flag in OperatorState? I was assuming we want to have it more fine-grained on OperatorSubtaskState. Maybe we can store the flag inside managed or raw state without changing the format?



On Fri, Dec 25, 2020 at 8:39 AM Yun Gao <yu...@aliyun.com> wrote:

   Hi all,

         I tested the previous PoC with the current tests and I found some new issues that might cause divergence, and sorry for there might also be some reversal for some previous problems:

     1. Which operators should wait for one more checkpoint before close ?

        One motivation for this FLIP is to ensure the 2PC sink commits the last part of data before closed, which makes the sink operator need to wait for one more checkpoint like onEndOfInput() -> waitForCheckpoint() -> notifyCheckpointComplete() -> close(). This lead to the issue which operators should wait for checkpoint? Possible options are 
                 a. Make all the operators (or UDF) implemented notifyCheckpointCompleted method wait for one more checkpoint. One exception is that since we can only snapshot one or all tasks for a legacy source operator to avoid data repetition[1], we could not support legacy operators and its chained operators to wait for checkpoints since there will be deadlock if part of the tasks are finished, this would finally be solved after legacy source are deprecated. The PoC used this option for now.
                b. Make operators (or UDF) implemented a special marker interface to wait for one more checkpoint.  


   2. Do we need to solve the case that tasks finished before triggered ?

      Previously I think we could postpone it, however, during testing I found that it might cause some problems since by default checkpoint failure would cause job failover, and the job would also need wait for another interval to trigger the next checkpoint. To pass the tests, I updated the PoC to include this part, and we may have a double think on if we need to include it or use some other options.

3. How to extend a new format for checkpoint meta ?

    Sorry previously I gave a wrong estimation, after I extract a sub-component for (de)serialize operator state, I found the problem just goes to the new OperatorStateSerializer. The problem seems to be that v2, v3 and v4 have different fields, thus they use different process when (de)serialize, which is a bit different from the case that we have a fixed steps and each step has different logic. Thus we might either
     a. Use base classes for each two version.
     b. Or have a unified framework contains all the possible fields across all version, and use empty field serializer to skip some fields in each version.

Best,
Yun

[1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-Option3.Allowtaskstofinish&Checkpointsdonotcontainthefinalstatesfromfinishedtasks

------------------------------------------------------------------
From:Yun Gao <yu...@aliyun.com.INVALID>
Send Time:2020 Dec. 16 (Wed.) 11:07
To:Aljoscha Krettek <al...@apache.org>; dev <de...@flink.apache.org>; user <us...@flink.apache.org>
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 "artificial" barriers for barrier 2 in case we receive  EndOfPartition while other inputs have already received barrier 2? I think that makes sense, yes.

      Yes, exactly, I would like to  insert "artificial" barriers for in case we receive  EndOfPartition while other inputs have already received barrier 2, and also for the similar cases that some input channels received EndOfPartition during checkpoint 2 is ongoing and when the task receive directly checkpoint triggering after all the precedent tasks are finished but not received their EndOfPartition yet.

     > 3. This indeed seems complex. Maybe we could switch to using composition instead of inheritance to make this more extensible?

    I re-checked the code and now I think composition would be better to avoid complex inheritance hierarchy by exposing the changed part `(de)serializeOperatorState` out, and I'll update the PoC to change this part. Very thanks for the suggestions!

   > 4. Don't we currently have the same problem? Even right now source tasks and non-source tasks behave differently when it comes to checkpoints. Are you saying we should fix that or would the new work introduce even 
more duplicate code?

  Currently since we would never trigger non-source tasks, thus the triggerCheckpoint logic is now implemented in the base StreamTask class and only be used by the source tasks. However, after the change the non-source tasks would also get triggered with a different behavior, we might not be able to continue using this pattern.

Best,
Yun


------------------------------------------------------------------
From:Aljoscha Krettek <al...@apache.org>
Send Time:2020 Dec. 15 (Tue.) 18:11
To:dev <de...@flink.apache.org>
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 the TM side, we now tend to decouple the logic for EndOfPartition with the normal alignment behaviors to avoid the complex interference (which seems to be a bit not trackable). We could do so by inserting suitable barriers for input channels received but not processed EndOfPartition. For example, if a task with four inputs has received barrier 2 from two input channels, but the other two inputs do not received barrier 2  before EndOfPartition due to the precedent tasks are finished, we could then insert barrier 2 for the last two channels so that we could still finish the checkpoint 2.

You mean we would insert "artificial" barriers for barrier 2 in case we 
receive  EndOfPartition while other inputs have already received barrier 
2? I think that makes sense, yes.

>      2. As we have discussed, if a tasks finished during we triggering the tasks, it would cause checkpoint failure and we should re-trigger its descendants. But if possible we think we might skip this issue at the first version to reduce the implementation complexity since it should not affect the correctness. We could considering support it in the following versions.

I think this should be completely fine.

>      3. We would have to add a field isFinished  to OperatorState so that we could not re-run finished sources after failover. However, this would require a new version of checkpoint meta. Currently Flink have an abstract MetaV2V3SerializerBase and have V2 and V3 extends it to share some implementation. To add V4 which is only different from V3 for one field, the current PoC want to introduce a new MetaV3V4SerializerBase extends MetaV2V3SerializerBase to share implementation between V3 and V4. This might looks a little complex and we might need a general mechanism to extend checkpoint meta format.

This indeed seems complex. Maybe we could switch to using composition 
instead of inheritance to make this more extensible?

>      4. With the change StreamTask would have two types of subclasses according to how to implement triggerCheckpoint, one is source tasks that perform checkpoints immediately and another is the non-source tasks that would notify CheckpointBarrierHandler in some way. However, since we have multiple source tasks (legacy and new source) and multiple non-source tasks (one-input, two-input, multiple-input), it would cause the cases that multiple subclasses share the same implementation and  cause code repetition. Currently the PoC introduces a new level of abstraction, namely SourceStreamTasks and NonSourceStreamTasks, but what makes it more complicated is that StreamingIterationHead extends OneInputStreamTask but it need to perform checkpoint as source tasks.

Don't we currently have the same problem? Even right now source tasks 
and non-source tasks behave differently when it comes to checkpoints. 
Are you saying we should fix that or would the new work introduce even 
more duplicate code?




-- 
Arvid Heise | Senior Java Developer

Follow us @VervericaData
--
Join Flink Forward - The Apache Flink Conference
Stream Processing | Event Driven | Real Time
--
Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
--Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji (Toni) Cheng


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

Posted by Yun Gao <yu...@aliyun.com>.
     Hi Avrid, 

         Very thanks for the feedbacks!

         For the second issue, sorry I think I might not make it very clear, I'm initially thinking the case that for example for a job with graph A -> B -> C, when we compute which tasks to trigger, A is still running, so we trigger A to start the checkpoint. However, before the triggering message reached A, A gets finished and the trigger message failed due to not found the task. In this case if we do not handle it, the checkpoint would failed due to timeout. However, by default failed checkpoint would cause job failure and we would also need to wait for a checkpoint interval for the next checkpoint. One solution would be check all the pending checkpoints to trigger B instead when JM is notified that A is finished.

       For the third issue, it should work if we store a special value for some filed in OperatorState or OperatorSubtaskState, for example, we might store a special subtaskState map inside the OperatorState to mark it is finished since the finished operator should always have an empty state. Very thanks for the advices! I'll try with this method. 

Best,
 Yun



------------------------------------------------------------------
From:Arvid Heise <ar...@ververica.com>
Send Time:2021 Jan. 5 (Tue.) 17:16
To:Yun Gao <yu...@aliyun.com>
Cc:Aljoscha Krettek <al...@apache.org>; dev <de...@flink.apache.org>; user <us...@flink.apache.org>
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. My gut feeling says that this is something we should only address for new sinks where we decouple the semantics of commits and checkpoints anyways. @Aljoscha Krettek any idea on this one?

2. I'm not sure I get it completely. Let's assume we have a source partition that is finished before the first checkpoint. Then, we would need to store the finished state of the subtask somehow. So I'm assuming, we still need to trigger some checkpointing code on finished subtasks.

3. Do we really want to store the finished flag in OperatorState? I was assuming we want to have it more fine-grained on OperatorSubtaskState. Maybe we can store the flag inside managed or raw state without changing the format?



On Fri, Dec 25, 2020 at 8:39 AM Yun Gao <yu...@aliyun.com> wrote:

   Hi all,

         I tested the previous PoC with the current tests and I found some new issues that might cause divergence, and sorry for there might also be some reversal for some previous problems:

     1. Which operators should wait for one more checkpoint before close ?

        One motivation for this FLIP is to ensure the 2PC sink commits the last part of data before closed, which makes the sink operator need to wait for one more checkpoint like onEndOfInput() -> waitForCheckpoint() -> notifyCheckpointComplete() -> close(). This lead to the issue which operators should wait for checkpoint? Possible options are 
                 a. Make all the operators (or UDF) implemented notifyCheckpointCompleted method wait for one more checkpoint. One exception is that since we can only snapshot one or all tasks for a legacy source operator to avoid data repetition[1], we could not support legacy operators and its chained operators to wait for checkpoints since there will be deadlock if part of the tasks are finished, this would finally be solved after legacy source are deprecated. The PoC used this option for now.
                b. Make operators (or UDF) implemented a special marker interface to wait for one more checkpoint.  


   2. Do we need to solve the case that tasks finished before triggered ?

      Previously I think we could postpone it, however, during testing I found that it might cause some problems since by default checkpoint failure would cause job failover, and the job would also need wait for another interval to trigger the next checkpoint. To pass the tests, I updated the PoC to include this part, and we may have a double think on if we need to include it or use some other options.

3. How to extend a new format for checkpoint meta ?

    Sorry previously I gave a wrong estimation, after I extract a sub-component for (de)serialize operator state, I found the problem just goes to the new OperatorStateSerializer. The problem seems to be that v2, v3 and v4 have different fields, thus they use different process when (de)serialize, which is a bit different from the case that we have a fixed steps and each step has different logic. Thus we might either
     a. Use base classes for each two version.
     b. Or have a unified framework contains all the possible fields across all version, and use empty field serializer to skip some fields in each version.

Best,
Yun

[1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-Option3.Allowtaskstofinish&Checkpointsdonotcontainthefinalstatesfromfinishedtasks

------------------------------------------------------------------
From:Yun Gao <yu...@aliyun.com.INVALID>
Send Time:2020 Dec. 16 (Wed.) 11:07
To:Aljoscha Krettek <al...@apache.org>; dev <de...@flink.apache.org>; user <us...@flink.apache.org>
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 "artificial" barriers for barrier 2 in case we receive  EndOfPartition while other inputs have already received barrier 2? I think that makes sense, yes.

      Yes, exactly, I would like to  insert "artificial" barriers for in case we receive  EndOfPartition while other inputs have already received barrier 2, and also for the similar cases that some input channels received EndOfPartition during checkpoint 2 is ongoing and when the task receive directly checkpoint triggering after all the precedent tasks are finished but not received their EndOfPartition yet.

     > 3. This indeed seems complex. Maybe we could switch to using composition instead of inheritance to make this more extensible?

    I re-checked the code and now I think composition would be better to avoid complex inheritance hierarchy by exposing the changed part `(de)serializeOperatorState` out, and I'll update the PoC to change this part. Very thanks for the suggestions!

   > 4. Don't we currently have the same problem? Even right now source tasks and non-source tasks behave differently when it comes to checkpoints. Are you saying we should fix that or would the new work introduce even 
more duplicate code?

  Currently since we would never trigger non-source tasks, thus the triggerCheckpoint logic is now implemented in the base StreamTask class and only be used by the source tasks. However, after the change the non-source tasks would also get triggered with a different behavior, we might not be able to continue using this pattern.

Best,
Yun


------------------------------------------------------------------
From:Aljoscha Krettek <al...@apache.org>
Send Time:2020 Dec. 15 (Tue.) 18:11
To:dev <de...@flink.apache.org>
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 the TM side, we now tend to decouple the logic for EndOfPartition with the normal alignment behaviors to avoid the complex interference (which seems to be a bit not trackable). We could do so by inserting suitable barriers for input channels received but not processed EndOfPartition. For example, if a task with four inputs has received barrier 2 from two input channels, but the other two inputs do not received barrier 2  before EndOfPartition due to the precedent tasks are finished, we could then insert barrier 2 for the last two channels so that we could still finish the checkpoint 2.

You mean we would insert "artificial" barriers for barrier 2 in case we 
receive  EndOfPartition while other inputs have already received barrier 
2? I think that makes sense, yes.

>      2. As we have discussed, if a tasks finished during we triggering the tasks, it would cause checkpoint failure and we should re-trigger its descendants. But if possible we think we might skip this issue at the first version to reduce the implementation complexity since it should not affect the correctness. We could considering support it in the following versions.

I think this should be completely fine.

>      3. We would have to add a field isFinished  to OperatorState so that we could not re-run finished sources after failover. However, this would require a new version of checkpoint meta. Currently Flink have an abstract MetaV2V3SerializerBase and have V2 and V3 extends it to share some implementation. To add V4 which is only different from V3 for one field, the current PoC want to introduce a new MetaV3V4SerializerBase extends MetaV2V3SerializerBase to share implementation between V3 and V4. This might looks a little complex and we might need a general mechanism to extend checkpoint meta format.

This indeed seems complex. Maybe we could switch to using composition 
instead of inheritance to make this more extensible?

>      4. With the change StreamTask would have two types of subclasses according to how to implement triggerCheckpoint, one is source tasks that perform checkpoints immediately and another is the non-source tasks that would notify CheckpointBarrierHandler in some way. However, since we have multiple source tasks (legacy and new source) and multiple non-source tasks (one-input, two-input, multiple-input), it would cause the cases that multiple subclasses share the same implementation and  cause code repetition. Currently the PoC introduces a new level of abstraction, namely SourceStreamTasks and NonSourceStreamTasks, but what makes it more complicated is that StreamingIterationHead extends OneInputStreamTask but it need to perform checkpoint as source tasks.

Don't we currently have the same problem? Even right now source tasks 
and non-source tasks behave differently when it comes to checkpoints. 
Are you saying we should fix that or would the new work introduce even 
more duplicate code?




-- 
Arvid Heise | Senior Java Developer

Follow us @VervericaData
--
Join Flink Forward - The Apache Flink Conference
Stream Processing | Event Driven | Real Time
--
Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
--Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji (Toni) Cheng


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

Posted by Arvid Heise <ar...@ververica.com>.
Hi Yun,

1. I'd think that this is an orthogonal issue, which I'd solve separately.
My gut feeling says that this is something we should only address for new
sinks where we decouple the semantics of commits and checkpoints
anyways. @Aljoscha
Krettek <al...@apache.org> any idea on this one?

2. I'm not sure I get it completely. Let's assume we have a source
partition that is finished before the first checkpoint. Then, we would need
to store the finished state of the subtask somehow. So I'm assuming, we
still need to trigger some checkpointing code on finished subtasks.

3. Do we really want to store the finished flag in OperatorState? I was
assuming we want to have it more fine-grained on OperatorSubtaskState.
Maybe we can store the flag inside managed or raw state without changing
the format?



On Fri, Dec 25, 2020 at 8:39 AM Yun Gao <yu...@aliyun.com> wrote:

>    Hi all,
>
>          I tested the previous PoC with the current tests and I found some
> new issues that might cause divergence, and sorry for there might also be
> some reversal for some previous problems:
>
>
>      1. Which operators should wait for one more checkpoint before close ?
>
>         One motivation for this FLIP is to ensure the 2PC sink commits the
> last part of data before closed, which makes the sink operator need to wait
> for one more checkpoint like onEndOfInput() -> waitForCheckpoint() ->
> notifyCheckpointComplete() -> close(). This lead to the issue which
> operators should wait for checkpoint? Possible options are
>                  a. Make all the operators (or UDF) implemented
> notifyCheckpointCompleted method wait for one more checkpoint. One
> exception is that since we can only snapshot one or all tasks for a legacy
> source operator to avoid data repetition[1], we could not support legacy
> operators and its chained operators to wait for checkpoints since there
> will be deadlock if part of the tasks are finished, this would finally be
> solved after legacy source are deprecated. The PoC used this option for now.
>                 b. Make operators (or UDF) implemented a special marker
> interface to wait for one more checkpoint.
>
>
>    2. Do we need to solve the case that tasks finished before triggered ?
>
>       Previously I think we could postpone it, however, during testing I
> found that it might cause some problems since by default checkpoint failure
> would cause job failover, and the job would also need wait for another
> interval to trigger the next checkpoint. To pass the tests, I updated the
> PoC to include this part, and we may have a double think on if we need to
> include it or use some other options.
>
> 3. How to extend a new format for checkpoint meta ?
>
>     Sorry previously I gave a wrong estimation, after I extract a
> sub-component for (de)serialize operator state, I found the problem just
> goes to the new OperatorStateSerializer. The problem seems to be that v2,
> v3 and v4 have different fields, thus they use different process when
> (de)serialize, which is a bit different from the case that we have a fixed
> steps and each step has different logic. Thus we might either
>      a. Use base classes for each two version.
>      b. Or have a unified framework contains all the possible fields
> across all version, and use empty field serializer to skip some fields in
> each version.
>
> Best,
> Yun
>
> [1]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-Option3.Allowtaskstofinish&Checkpointsdonotcontainthefinalstatesfromfinishedtasks
>
> ------------------------------------------------------------------
> From:Yun Gao <yu...@aliyun.com.INVALID>
> Send Time:2020 Dec. 16 (Wed.) 11:07
> To:Aljoscha Krettek <al...@apache.org>; dev <de...@flink.apache.org>;
> user <us...@flink.apache.org>
> 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 "artificial" barriers for barrier 2 in case we receive  EndOfPartition while other inputs have already received barrier 2? I think that makes sense, yes.
>
>
>       Yes, exactly, I would like to  insert "artificial" barriers for in case we receive  EndOfPartition while other inputs have already received barrier 2, and also for the similar cases that some input channels received EndOfPartition during checkpoint 2 is ongoing and when the task receive directly checkpoint triggering after all the precedent tasks are finished but not received their EndOfPartition yet.
>
>
>      > 3. This indeed seems complex. Maybe we could switch to using composition instead of inheritance to make this more extensible?
>
>
>     I re-checked the code and now I think composition would be better to avoid complex inheritance hierarchy by exposing the changed part `(de)serializeOperatorState` out, and I'll update the PoC to change this part. Very thanks for the suggestions!
>
>
>    > 4. Don't we currently have the same problem? Even right now source tasks and non-source tasks behave differently when it comes to checkpoints. Are you saying we should fix that or would the new work introduce even
> more duplicate code?
>
>
>   Currently since we would never trigger non-source tasks, thus the triggerCheckpoint logic is now implemented in the base StreamTask class and only be used by the source tasks. However, after the change the non-source tasks would also get triggered with a different behavior, we might not be able to continue using this pattern.
>
> Best,
> Yun
>
>
> ------------------------------------------------------------------
> From:Aljoscha Krettek <al...@apache.org>
> Send Time:2020 Dec. 15 (Tue.) 18:11
> To:dev <de...@flink.apache.org>
> 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 the TM side, we now tend to decouple the logic for EndOfPartition with the normal alignment behaviors to avoid the complex interference (which seems to be a bit not trackable). We could do so by inserting suitable barriers for input channels received but not processed EndOfPartition. For example, if a task with four inputs has received barrier 2 from two input channels, but the other two inputs do not received barrier 2  before EndOfPartition due to the precedent tasks are finished, we could then insert barrier 2 for the last two channels so that we could still finish the checkpoint 2.
>
> You mean we would insert "artificial" barriers for barrier 2 in case we
> receive  EndOfPartition while other inputs have already received barrier
> 2? I think that makes sense, yes.
>
>
> >      2. As we have discussed, if a tasks finished during we triggering the tasks, it would cause checkpoint failure and we should re-trigger its descendants. But if possible we think we might skip this issue at the first version to reduce the implementation complexity since it should not affect the correctness. We could considering support it in the following versions.
>
> I think this should be completely fine.
>
>
> >      3. We would have to add a field isFinished  to OperatorState so that we could not re-run finished sources after failover. However, this would require a new version of checkpoint meta. Currently Flink have an abstract MetaV2V3SerializerBase and have V2 and V3 extends it to share some implementation. To add V4 which is only different from V3 for one field, the current PoC want to introduce a new MetaV3V4SerializerBase extends MetaV2V3SerializerBase to share implementation between V3 and V4. This might looks a little complex and we might need a general mechanism to extend checkpoint meta format.
>
> This indeed seems complex. Maybe we could switch to using composition
> instead of inheritance to make this more extensible?
>
>
> >      4. With the change StreamTask would have two types of subclasses according to how to implement triggerCheckpoint, one is source tasks that perform checkpoints immediately and another is the non-source tasks that would notify CheckpointBarrierHandler in some way. However, since we have multiple source tasks (legacy and new source) and multiple non-source tasks (one-input, two-input, multiple-input), it would cause the cases that multiple subclasses share the same implementation and  cause code repetition. Currently the PoC introduces a new level of abstraction, namely SourceStreamTasks and NonSourceStreamTasks, but what makes it more complicated is that StreamingIterationHead extends OneInputStreamTask but it need to perform checkpoint as source tasks.
>
> Don't we currently have the same problem? Even right now source tasks
> and non-source tasks behave differently when it comes to checkpoints.
> Are you saying we should fix that or would the new work introduce even
> more duplicate code?
>
>
>

-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

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

Posted by Arvid Heise <ar...@ververica.com>.
Hi Yun,

1. I'd think that this is an orthogonal issue, which I'd solve separately.
My gut feeling says that this is something we should only address for new
sinks where we decouple the semantics of commits and checkpoints
anyways. @Aljoscha
Krettek <al...@apache.org> any idea on this one?

2. I'm not sure I get it completely. Let's assume we have a source
partition that is finished before the first checkpoint. Then, we would need
to store the finished state of the subtask somehow. So I'm assuming, we
still need to trigger some checkpointing code on finished subtasks.

3. Do we really want to store the finished flag in OperatorState? I was
assuming we want to have it more fine-grained on OperatorSubtaskState.
Maybe we can store the flag inside managed or raw state without changing
the format?



On Fri, Dec 25, 2020 at 8:39 AM Yun Gao <yu...@aliyun.com> wrote:

>    Hi all,
>
>          I tested the previous PoC with the current tests and I found some
> new issues that might cause divergence, and sorry for there might also be
> some reversal for some previous problems:
>
>
>      1. Which operators should wait for one more checkpoint before close ?
>
>         One motivation for this FLIP is to ensure the 2PC sink commits the
> last part of data before closed, which makes the sink operator need to wait
> for one more checkpoint like onEndOfInput() -> waitForCheckpoint() ->
> notifyCheckpointComplete() -> close(). This lead to the issue which
> operators should wait for checkpoint? Possible options are
>                  a. Make all the operators (or UDF) implemented
> notifyCheckpointCompleted method wait for one more checkpoint. One
> exception is that since we can only snapshot one or all tasks for a legacy
> source operator to avoid data repetition[1], we could not support legacy
> operators and its chained operators to wait for checkpoints since there
> will be deadlock if part of the tasks are finished, this would finally be
> solved after legacy source are deprecated. The PoC used this option for now.
>                 b. Make operators (or UDF) implemented a special marker
> interface to wait for one more checkpoint.
>
>
>    2. Do we need to solve the case that tasks finished before triggered ?
>
>       Previously I think we could postpone it, however, during testing I
> found that it might cause some problems since by default checkpoint failure
> would cause job failover, and the job would also need wait for another
> interval to trigger the next checkpoint. To pass the tests, I updated the
> PoC to include this part, and we may have a double think on if we need to
> include it or use some other options.
>
> 3. How to extend a new format for checkpoint meta ?
>
>     Sorry previously I gave a wrong estimation, after I extract a
> sub-component for (de)serialize operator state, I found the problem just
> goes to the new OperatorStateSerializer. The problem seems to be that v2,
> v3 and v4 have different fields, thus they use different process when
> (de)serialize, which is a bit different from the case that we have a fixed
> steps and each step has different logic. Thus we might either
>      a. Use base classes for each two version.
>      b. Or have a unified framework contains all the possible fields
> across all version, and use empty field serializer to skip some fields in
> each version.
>
> Best,
> Yun
>
> [1]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-Option3.Allowtaskstofinish&Checkpointsdonotcontainthefinalstatesfromfinishedtasks
>
> ------------------------------------------------------------------
> From:Yun Gao <yu...@aliyun.com.INVALID>
> Send Time:2020 Dec. 16 (Wed.) 11:07
> To:Aljoscha Krettek <al...@apache.org>; dev <de...@flink.apache.org>;
> user <us...@flink.apache.org>
> 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 "artificial" barriers for barrier 2 in case we receive  EndOfPartition while other inputs have already received barrier 2? I think that makes sense, yes.
>
>
>       Yes, exactly, I would like to  insert "artificial" barriers for in case we receive  EndOfPartition while other inputs have already received barrier 2, and also for the similar cases that some input channels received EndOfPartition during checkpoint 2 is ongoing and when the task receive directly checkpoint triggering after all the precedent tasks are finished but not received their EndOfPartition yet.
>
>
>      > 3. This indeed seems complex. Maybe we could switch to using composition instead of inheritance to make this more extensible?
>
>
>     I re-checked the code and now I think composition would be better to avoid complex inheritance hierarchy by exposing the changed part `(de)serializeOperatorState` out, and I'll update the PoC to change this part. Very thanks for the suggestions!
>
>
>    > 4. Don't we currently have the same problem? Even right now source tasks and non-source tasks behave differently when it comes to checkpoints. Are you saying we should fix that or would the new work introduce even
> more duplicate code?
>
>
>   Currently since we would never trigger non-source tasks, thus the triggerCheckpoint logic is now implemented in the base StreamTask class and only be used by the source tasks. However, after the change the non-source tasks would also get triggered with a different behavior, we might not be able to continue using this pattern.
>
> Best,
> Yun
>
>
> ------------------------------------------------------------------
> From:Aljoscha Krettek <al...@apache.org>
> Send Time:2020 Dec. 15 (Tue.) 18:11
> To:dev <de...@flink.apache.org>
> 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 the TM side, we now tend to decouple the logic for EndOfPartition with the normal alignment behaviors to avoid the complex interference (which seems to be a bit not trackable). We could do so by inserting suitable barriers for input channels received but not processed EndOfPartition. For example, if a task with four inputs has received barrier 2 from two input channels, but the other two inputs do not received barrier 2  before EndOfPartition due to the precedent tasks are finished, we could then insert barrier 2 for the last two channels so that we could still finish the checkpoint 2.
>
> You mean we would insert "artificial" barriers for barrier 2 in case we
> receive  EndOfPartition while other inputs have already received barrier
> 2? I think that makes sense, yes.
>
>
> >      2. As we have discussed, if a tasks finished during we triggering the tasks, it would cause checkpoint failure and we should re-trigger its descendants. But if possible we think we might skip this issue at the first version to reduce the implementation complexity since it should not affect the correctness. We could considering support it in the following versions.
>
> I think this should be completely fine.
>
>
> >      3. We would have to add a field isFinished  to OperatorState so that we could not re-run finished sources after failover. However, this would require a new version of checkpoint meta. Currently Flink have an abstract MetaV2V3SerializerBase and have V2 and V3 extends it to share some implementation. To add V4 which is only different from V3 for one field, the current PoC want to introduce a new MetaV3V4SerializerBase extends MetaV2V3SerializerBase to share implementation between V3 and V4. This might looks a little complex and we might need a general mechanism to extend checkpoint meta format.
>
> This indeed seems complex. Maybe we could switch to using composition
> instead of inheritance to make this more extensible?
>
>
> >      4. With the change StreamTask would have two types of subclasses according to how to implement triggerCheckpoint, one is source tasks that perform checkpoints immediately and another is the non-source tasks that would notify CheckpointBarrierHandler in some way. However, since we have multiple source tasks (legacy and new source) and multiple non-source tasks (one-input, two-input, multiple-input), it would cause the cases that multiple subclasses share the same implementation and  cause code repetition. Currently the PoC introduces a new level of abstraction, namely SourceStreamTasks and NonSourceStreamTasks, but what makes it more complicated is that StreamingIterationHead extends OneInputStreamTask but it need to perform checkpoint as source tasks.
>
> Don't we currently have the same problem? Even right now source tasks
> and non-source tasks behave differently when it comes to checkpoints.
> Are you saying we should fix that or would the new work introduce even
> more duplicate code?
>
>
>

-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
   Hi all,

         I tested the previous PoC with the current tests and I found some new issues that might cause divergence, and sorry for there might also be some reversal for some previous problems:

     1. Which operators should wait for one more checkpoint before close ?

        One motivation for this FLIP is to ensure the 2PC sink commits the last part of data before closed, which makes the sink operator need to wait for one more checkpoint like onEndOfInput() -> waitForCheckpoint() -> notifyCheckpointComplete() -> close(). This lead to the issue which operators should wait for checkpoint? Possible options are 
                 a. Make all the operators (or UDF) implemented notifyCheckpointCompleted method wait for one more checkpoint. One exception is that since we can only snapshot one or all tasks for a legacy source operator to avoid data repetition[1], we could not support legacy operators and its chained operators to wait for checkpoints since there will be deadlock if part of the tasks are finished, this would finally be solved after legacy source are deprecated. The PoC used this option for now.
                b. Make operators (or UDF) implemented a special marker interface to wait for one more checkpoint.  


   2. Do we need to solve the case that tasks finished before triggered ?

      Previously I think we could postpone it, however, during testing I found that it might cause some problems since by default checkpoint failure would cause job failover, and the job would also need wait for another interval to trigger the next checkpoint. To pass the tests, I updated the PoC to include this part, and we may have a double think on if we need to include it or use some other options.

3. How to extend a new format for checkpoint meta ?

    Sorry previously I gave a wrong estimation, after I extract a sub-component for (de)serialize operator state, I found the problem just goes to the new OperatorStateSerializer. The problem seems to be that v2, v3 and v4 have different fields, thus they use different process when (de)serialize, which is a bit different from the case that we have a fixed steps and each step has different logic. Thus we might either
     a. Use base classes for each two version.
     b. Or have a unified framework contains all the possible fields across all version, and use empty field serializer to skip some fields in each version.

Best,
Yun

[1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-Option3.Allowtaskstofinish&Checkpointsdonotcontainthefinalstatesfromfinishedtasks


------------------------------------------------------------------
From:Yun Gao <yu...@aliyun.com.INVALID>
Send Time:2020 Dec. 16 (Wed.) 11:07
To:Aljoscha Krettek <al...@apache.org>; dev <de...@flink.apache.org>; user <us...@flink.apache.org>
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 "artificial" barriers for barrier 2 in case we receive  EndOfPartition while other inputs have already received barrier 2? I think that makes sense, yes.

      Yes, exactly, I would like to  insert "artificial" barriers for in case we receive  EndOfPartition while other inputs have already received barrier 2, and also for the similar cases that some input channels received EndOfPartition during checkpoint 2 is ongoing and when the task receive directly checkpoint triggering after all the precedent tasks are finished but not received their EndOfPartition yet.

     > 3. This indeed seems complex. Maybe we could switch to using composition instead of inheritance to make this more extensible?

    I re-checked the code and now I think composition would be better to avoid complex inheritance hierarchy by exposing the changed part `(de)serializeOperatorState` out, and I'll update the PoC to change this part. Very thanks for the suggestions!

   > 4. Don't we currently have the same problem? Even right now source tasks and non-source tasks behave differently when it comes to checkpoints. Are you saying we should fix that or would the new work introduce even 
more duplicate code?

  Currently since we would never trigger non-source tasks, thus the triggerCheckpoint logic is now implemented in the base StreamTask class and only be used by the source tasks. However, after the change the non-source tasks would also get triggered with a different behavior, we might not be able to continue using this pattern.

Best,
Yun


------------------------------------------------------------------
From:Aljoscha Krettek <al...@apache.org>
Send Time:2020 Dec. 15 (Tue.) 18:11
To:dev <de...@flink.apache.org>
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 the TM side, we now tend to decouple the logic for EndOfPartition with the normal alignment behaviors to avoid the complex interference (which seems to be a bit not trackable). We could do so by inserting suitable barriers for input channels received but not processed EndOfPartition. For example, if a task with four inputs has received barrier 2 from two input channels, but the other two inputs do not received barrier 2  before EndOfPartition due to the precedent tasks are finished, we could then insert barrier 2 for the last two channels so that we could still finish the checkpoint 2.

You mean we would insert "artificial" barriers for barrier 2 in case we 
receive  EndOfPartition while other inputs have already received barrier 
2? I think that makes sense, yes.

>      2. As we have discussed, if a tasks finished during we triggering the tasks, it would cause checkpoint failure and we should re-trigger its descendants. But if possible we think we might skip this issue at the first version to reduce the implementation complexity since it should not affect the correctness. We could considering support it in the following versions.

I think this should be completely fine.

>      3. We would have to add a field isFinished  to OperatorState so that we could not re-run finished sources after failover. However, this would require a new version of checkpoint meta. Currently Flink have an abstract MetaV2V3SerializerBase and have V2 and V3 extends it to share some implementation. To add V4 which is only different from V3 for one field, the current PoC want to introduce a new MetaV3V4SerializerBase extends MetaV2V3SerializerBase to share implementation between V3 and V4. This might looks a little complex and we might need a general mechanism to extend checkpoint meta format.

This indeed seems complex. Maybe we could switch to using composition 
instead of inheritance to make this more extensible?

>      4. With the change StreamTask would have two types of subclasses according to how to implement triggerCheckpoint, one is source tasks that perform checkpoints immediately and another is the non-source tasks that would notify CheckpointBarrierHandler in some way. However, since we have multiple source tasks (legacy and new source) and multiple non-source tasks (one-input, two-input, multiple-input), it would cause the cases that multiple subclasses share the same implementation and  cause code repetition. Currently the PoC introduces a new level of abstraction, namely SourceStreamTasks and NonSourceStreamTasks, but what makes it more complicated is that StreamingIterationHead extends OneInputStreamTask but it need to perform checkpoint as source tasks.

Don't we currently have the same problem? Even right now source tasks 
and non-source tasks behave differently when it comes to checkpoints. 
Are you saying we should fix that or would the new work introduce even 
more duplicate code?



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

Posted by Yun Gao <yu...@aliyun.com>.
   Hi all,

         I tested the previous PoC with the current tests and I found some new issues that might cause divergence, and sorry for there might also be some reversal for some previous problems:

     1. Which operators should wait for one more checkpoint before close ?

        One motivation for this FLIP is to ensure the 2PC sink commits the last part of data before closed, which makes the sink operator need to wait for one more checkpoint like onEndOfInput() -> waitForCheckpoint() -> notifyCheckpointComplete() -> close(). This lead to the issue which operators should wait for checkpoint? Possible options are 
                 a. Make all the operators (or UDF) implemented notifyCheckpointCompleted method wait for one more checkpoint. One exception is that since we can only snapshot one or all tasks for a legacy source operator to avoid data repetition[1], we could not support legacy operators and its chained operators to wait for checkpoints since there will be deadlock if part of the tasks are finished, this would finally be solved after legacy source are deprecated. The PoC used this option for now.
                b. Make operators (or UDF) implemented a special marker interface to wait for one more checkpoint.  


   2. Do we need to solve the case that tasks finished before triggered ?

      Previously I think we could postpone it, however, during testing I found that it might cause some problems since by default checkpoint failure would cause job failover, and the job would also need wait for another interval to trigger the next checkpoint. To pass the tests, I updated the PoC to include this part, and we may have a double think on if we need to include it or use some other options.

3. How to extend a new format for checkpoint meta ?

    Sorry previously I gave a wrong estimation, after I extract a sub-component for (de)serialize operator state, I found the problem just goes to the new OperatorStateSerializer. The problem seems to be that v2, v3 and v4 have different fields, thus they use different process when (de)serialize, which is a bit different from the case that we have a fixed steps and each step has different logic. Thus we might either
     a. Use base classes for each two version.
     b. Or have a unified framework contains all the possible fields across all version, and use empty field serializer to skip some fields in each version.

Best,
Yun

[1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-Option3.Allowtaskstofinish&Checkpointsdonotcontainthefinalstatesfromfinishedtasks


------------------------------------------------------------------
From:Yun Gao <yu...@aliyun.com.INVALID>
Send Time:2020 Dec. 16 (Wed.) 11:07
To:Aljoscha Krettek <al...@apache.org>; dev <de...@flink.apache.org>; user <us...@flink.apache.org>
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 "artificial" barriers for barrier 2 in case we receive  EndOfPartition while other inputs have already received barrier 2? I think that makes sense, yes.

      Yes, exactly, I would like to  insert "artificial" barriers for in case we receive  EndOfPartition while other inputs have already received barrier 2, and also for the similar cases that some input channels received EndOfPartition during checkpoint 2 is ongoing and when the task receive directly checkpoint triggering after all the precedent tasks are finished but not received their EndOfPartition yet.

     > 3. This indeed seems complex. Maybe we could switch to using composition instead of inheritance to make this more extensible?

    I re-checked the code and now I think composition would be better to avoid complex inheritance hierarchy by exposing the changed part `(de)serializeOperatorState` out, and I'll update the PoC to change this part. Very thanks for the suggestions!

   > 4. Don't we currently have the same problem? Even right now source tasks and non-source tasks behave differently when it comes to checkpoints. Are you saying we should fix that or would the new work introduce even 
more duplicate code?

  Currently since we would never trigger non-source tasks, thus the triggerCheckpoint logic is now implemented in the base StreamTask class and only be used by the source tasks. However, after the change the non-source tasks would also get triggered with a different behavior, we might not be able to continue using this pattern.

Best,
Yun


------------------------------------------------------------------
From:Aljoscha Krettek <al...@apache.org>
Send Time:2020 Dec. 15 (Tue.) 18:11
To:dev <de...@flink.apache.org>
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 the TM side, we now tend to decouple the logic for EndOfPartition with the normal alignment behaviors to avoid the complex interference (which seems to be a bit not trackable). We could do so by inserting suitable barriers for input channels received but not processed EndOfPartition. For example, if a task with four inputs has received barrier 2 from two input channels, but the other two inputs do not received barrier 2  before EndOfPartition due to the precedent tasks are finished, we could then insert barrier 2 for the last two channels so that we could still finish the checkpoint 2.

You mean we would insert "artificial" barriers for barrier 2 in case we 
receive  EndOfPartition while other inputs have already received barrier 
2? I think that makes sense, yes.

>      2. As we have discussed, if a tasks finished during we triggering the tasks, it would cause checkpoint failure and we should re-trigger its descendants. But if possible we think we might skip this issue at the first version to reduce the implementation complexity since it should not affect the correctness. We could considering support it in the following versions.

I think this should be completely fine.

>      3. We would have to add a field isFinished  to OperatorState so that we could not re-run finished sources after failover. However, this would require a new version of checkpoint meta. Currently Flink have an abstract MetaV2V3SerializerBase and have V2 and V3 extends it to share some implementation. To add V4 which is only different from V3 for one field, the current PoC want to introduce a new MetaV3V4SerializerBase extends MetaV2V3SerializerBase to share implementation between V3 and V4. This might looks a little complex and we might need a general mechanism to extend checkpoint meta format.

This indeed seems complex. Maybe we could switch to using composition 
instead of inheritance to make this more extensible?

>      4. With the change StreamTask would have two types of subclasses according to how to implement triggerCheckpoint, one is source tasks that perform checkpoints immediately and another is the non-source tasks that would notify CheckpointBarrierHandler in some way. However, since we have multiple source tasks (legacy and new source) and multiple non-source tasks (one-input, two-input, multiple-input), it would cause the cases that multiple subclasses share the same implementation and  cause code repetition. Currently the PoC introduces a new level of abstraction, namely SourceStreamTasks and NonSourceStreamTasks, but what makes it more complicated is that StreamingIterationHead extends OneInputStreamTask but it need to perform checkpoint as source tasks.

Don't we currently have the same problem? Even right now source tasks 
and non-source tasks behave differently when it comes to checkpoints. 
Are you saying we should fix that or would the new work introduce even 
more duplicate code?



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

Posted by Yun Gao <yu...@aliyun.com>.
     Hi Aljoscha,

        Very thanks for the feedbacks! For the remaining issues:

      > 1. You mean we would insert "artificial" barriers for barrier 2 in case we receive  EndOfPartition while other inputs have already received barrier 2? I think that makes sense, yes.

      Yes, exactly, I would like to  insert "artificial" barriers for in case we receive  EndOfPartition while other inputs have already received barrier 2, and also for the similar cases that some input channels received EndOfPartition during checkpoint 2 is ongoing and when the task receive directly checkpoint triggering after all the precedent tasks are finished but not received their EndOfPartition yet.

     > 3. This indeed seems complex. Maybe we could switch to using composition instead of inheritance to make this more extensible?

    I re-checked the code and now I think composition would be better to avoid complex inheritance hierarchy by exposing the changed part `(de)serializeOperatorState` out, and I'll update the PoC to change this part. Very thanks for the suggestions!

   > 4. Don't we currently have the same problem? Even right now source tasks and non-source tasks behave differently when it comes to checkpoints. Are you saying we should fix that or would the new work introduce even 
more duplicate code?

  Currently since we would never trigger non-source tasks, thus the triggerCheckpoint logic is now implemented in the base StreamTask class and only be used by the source tasks. However, after the change the non-source tasks would also get triggered with a different behavior, we might not be able to continue using this pattern.

Best,
Yun


------------------------------------------------------------------
From:Aljoscha Krettek <al...@apache.org>
Send Time:2020 Dec. 15 (Tue.) 18:11
To:dev <de...@flink.apache.org>
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 the TM side, we now tend to decouple the logic for EndOfPartition with the normal alignment behaviors to avoid the complex interference (which seems to be a bit not trackable). We could do so by inserting suitable barriers for input channels received but not processed EndOfPartition. For example, if a task with four inputs has received barrier 2 from two input channels, but the other two inputs do not received barrier 2  before EndOfPartition due to the precedent tasks are finished, we could then insert barrier 2 for the last two channels so that we could still finish the checkpoint 2.

You mean we would insert "artificial" barriers for barrier 2 in case we 
receive  EndOfPartition while other inputs have already received barrier 
2? I think that makes sense, yes.

>      2. As we have discussed, if a tasks finished during we triggering the tasks, it would cause checkpoint failure and we should re-trigger its descendants. But if possible we think we might skip this issue at the first version to reduce the implementation complexity since it should not affect the correctness. We could considering support it in the following versions.

I think this should be completely fine.

>      3. We would have to add a field isFinished  to OperatorState so that we could not re-run finished sources after failover. However, this would require a new version of checkpoint meta. Currently Flink have an abstract MetaV2V3SerializerBase and have V2 and V3 extends it to share some implementation. To add V4 which is only different from V3 for one field, the current PoC want to introduce a new MetaV3V4SerializerBase extends MetaV2V3SerializerBase to share implementation between V3 and V4. This might looks a little complex and we might need a general mechanism to extend checkpoint meta format.

This indeed seems complex. Maybe we could switch to using composition 
instead of inheritance to make this more extensible?

>      4. With the change StreamTask would have two types of subclasses according to how to implement triggerCheckpoint, one is source tasks that perform checkpoints immediately and another is the non-source tasks that would notify CheckpointBarrierHandler in some way. However, since we have multiple source tasks (legacy and new source) and multiple non-source tasks (one-input, two-input, multiple-input), it would cause the cases that multiple subclasses share the same implementation and  cause code repetition. Currently the PoC introduces a new level of abstraction, namely SourceStreamTasks and NonSourceStreamTasks, but what makes it more complicated is that StreamingIterationHead extends OneInputStreamTask but it need to perform checkpoint as source tasks.

Don't we currently have the same problem? Even right now source tasks 
and non-source tasks behave differently when it comes to checkpoints. 
Are you saying we should fix that or would the new work introduce even 
more duplicate code?


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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
     Hi Aljoscha,

        Very thanks for the feedbacks! For the remaining issues:

      > 1. You mean we would insert "artificial" barriers for barrier 2 in case we receive  EndOfPartition while other inputs have already received barrier 2? I think that makes sense, yes.

      Yes, exactly, I would like to  insert "artificial" barriers for in case we receive  EndOfPartition while other inputs have already received barrier 2, and also for the similar cases that some input channels received EndOfPartition during checkpoint 2 is ongoing and when the task receive directly checkpoint triggering after all the precedent tasks are finished but not received their EndOfPartition yet.

     > 3. This indeed seems complex. Maybe we could switch to using composition instead of inheritance to make this more extensible?

    I re-checked the code and now I think composition would be better to avoid complex inheritance hierarchy by exposing the changed part `(de)serializeOperatorState` out, and I'll update the PoC to change this part. Very thanks for the suggestions!

   > 4. Don't we currently have the same problem? Even right now source tasks and non-source tasks behave differently when it comes to checkpoints. Are you saying we should fix that or would the new work introduce even 
more duplicate code?

  Currently since we would never trigger non-source tasks, thus the triggerCheckpoint logic is now implemented in the base StreamTask class and only be used by the source tasks. However, after the change the non-source tasks would also get triggered with a different behavior, we might not be able to continue using this pattern.

Best,
Yun


------------------------------------------------------------------
From:Aljoscha Krettek <al...@apache.org>
Send Time:2020 Dec. 15 (Tue.) 18:11
To:dev <de...@flink.apache.org>
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 the TM side, we now tend to decouple the logic for EndOfPartition with the normal alignment behaviors to avoid the complex interference (which seems to be a bit not trackable). We could do so by inserting suitable barriers for input channels received but not processed EndOfPartition. For example, if a task with four inputs has received barrier 2 from two input channels, but the other two inputs do not received barrier 2  before EndOfPartition due to the precedent tasks are finished, we could then insert barrier 2 for the last two channels so that we could still finish the checkpoint 2.

You mean we would insert "artificial" barriers for barrier 2 in case we 
receive  EndOfPartition while other inputs have already received barrier 
2? I think that makes sense, yes.

>      2. As we have discussed, if a tasks finished during we triggering the tasks, it would cause checkpoint failure and we should re-trigger its descendants. But if possible we think we might skip this issue at the first version to reduce the implementation complexity since it should not affect the correctness. We could considering support it in the following versions.

I think this should be completely fine.

>      3. We would have to add a field isFinished  to OperatorState so that we could not re-run finished sources after failover. However, this would require a new version of checkpoint meta. Currently Flink have an abstract MetaV2V3SerializerBase and have V2 and V3 extends it to share some implementation. To add V4 which is only different from V3 for one field, the current PoC want to introduce a new MetaV3V4SerializerBase extends MetaV2V3SerializerBase to share implementation between V3 and V4. This might looks a little complex and we might need a general mechanism to extend checkpoint meta format.

This indeed seems complex. Maybe we could switch to using composition 
instead of inheritance to make this more extensible?

>      4. With the change StreamTask would have two types of subclasses according to how to implement triggerCheckpoint, one is source tasks that perform checkpoints immediately and another is the non-source tasks that would notify CheckpointBarrierHandler in some way. However, since we have multiple source tasks (legacy and new source) and multiple non-source tasks (one-input, two-input, multiple-input), it would cause the cases that multiple subclasses share the same implementation and  cause code repetition. Currently the PoC introduces a new level of abstraction, namely SourceStreamTasks and NonSourceStreamTasks, but what makes it more complicated is that StreamingIterationHead extends OneInputStreamTask but it need to perform checkpoint as source tasks.

Don't we currently have the same problem? Even right now source tasks 
and non-source tasks behave differently when it comes to checkpoints. 
Are you saying we should fix that or would the new work introduce even 
more duplicate code?


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

Posted by Aljoscha Krettek <al...@apache.org>.
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 interference (which seems to be a bit not trackable). We could do so by inserting suitable barriers for input channels received but not processed EndOfPartition. For example, if a task with four inputs has received barrier 2 from two input channels, but the other two inputs do not received barrier 2  before EndOfPartition due to the precedent tasks are finished, we could then insert barrier 2 for the last two channels so that we could still finish the checkpoint 2.

You mean we would insert "artificial" barriers for barrier 2 in case we 
receive  EndOfPartition while other inputs have already received barrier 
2? I think that makes sense, yes.

>      2. As we have discussed, if a tasks finished during we triggering the tasks, it would cause checkpoint failure and we should re-trigger its descendants. But if possible we think we might skip this issue at the first version to reduce the implementation complexity since it should not affect the correctness. We could considering support it in the following versions.

I think this should be completely fine.

>      3. We would have to add a field isFinished  to OperatorState so that we could not re-run finished sources after failover. However, this would require a new version of checkpoint meta. Currently Flink have an abstract MetaV2V3SerializerBase and have V2 and V3 extends it to share some implementation. To add V4 which is only different from V3 for one field, the current PoC want to introduce a new MetaV3V4SerializerBase extends MetaV2V3SerializerBase to share implementation between V3 and V4. This might looks a little complex and we might need a general mechanism to extend checkpoint meta format.

This indeed seems complex. Maybe we could switch to using composition 
instead of inheritance to make this more extensible?

>      4. With the change StreamTask would have two types of subclasses according to how to implement triggerCheckpoint, one is source tasks that perform checkpoints immediately and another is the non-source tasks that would notify CheckpointBarrierHandler in some way. However, since we have multiple source tasks (legacy and new source) and multiple non-source tasks (one-input, two-input, multiple-input), it would cause the cases that multiple subclasses share the same implementation and  cause code repetition. Currently the PoC introduces a new level of abstraction, namely SourceStreamTasks and NonSourceStreamTasks, but what makes it more complicated is that StreamingIterationHead extends OneInputStreamTask but it need to perform checkpoint as source tasks.

Don't we currently have the same problem? Even right now source tasks 
and non-source tasks behave differently when it comes to checkpoints. 
Are you saying we should fix that or would the new work introduce even 
more duplicate code?

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

Posted by Aljoscha Krettek <al...@apache.org>.
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 interference (which seems to be a bit not trackable). We could do so by inserting suitable barriers for input channels received but not processed EndOfPartition. For example, if a task with four inputs has received barrier 2 from two input channels, but the other two inputs do not received barrier 2  before EndOfPartition due to the precedent tasks are finished, we could then insert barrier 2 for the last two channels so that we could still finish the checkpoint 2.

You mean we would insert "artificial" barriers for barrier 2 in case we 
receive  EndOfPartition while other inputs have already received barrier 
2? I think that makes sense, yes.

>      2. As we have discussed, if a tasks finished during we triggering the tasks, it would cause checkpoint failure and we should re-trigger its descendants. But if possible we think we might skip this issue at the first version to reduce the implementation complexity since it should not affect the correctness. We could considering support it in the following versions.

I think this should be completely fine.

>      3. We would have to add a field isFinished  to OperatorState so that we could not re-run finished sources after failover. However, this would require a new version of checkpoint meta. Currently Flink have an abstract MetaV2V3SerializerBase and have V2 and V3 extends it to share some implementation. To add V4 which is only different from V3 for one field, the current PoC want to introduce a new MetaV3V4SerializerBase extends MetaV2V3SerializerBase to share implementation between V3 and V4. This might looks a little complex and we might need a general mechanism to extend checkpoint meta format.

This indeed seems complex. Maybe we could switch to using composition 
instead of inheritance to make this more extensible?

>      4. With the change StreamTask would have two types of subclasses according to how to implement triggerCheckpoint, one is source tasks that perform checkpoints immediately and another is the non-source tasks that would notify CheckpointBarrierHandler in some way. However, since we have multiple source tasks (legacy and new source) and multiple non-source tasks (one-input, two-input, multiple-input), it would cause the cases that multiple subclasses share the same implementation and  cause code repetition. Currently the PoC introduces a new level of abstraction, namely SourceStreamTasks and NonSourceStreamTasks, but what makes it more complicated is that StreamingIterationHead extends OneInputStreamTask but it need to perform checkpoint as source tasks.

Don't we currently have the same problem? Even right now source tasks 
and non-source tasks behave differently when it comes to checkpoints. 
Are you saying we should fix that or would the new work introduce even 
more duplicate code?


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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
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 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 interference (which seems to be a bit not trackable). We could do so by inserting suitable barriers for input channels received but not processed EndOfPartition. For example, if a task with four inputs has received barrier 2 from two input channels, but the other two inputs do not received barrier 2  before EndOfPartition due to the precedent tasks are finished, we could then insert barrier 2 for the last two channels so that we could still finish the checkpoint 2.
    2. As we have discussed, if a tasks finished during we triggering the tasks, it would cause checkpoint failure and we should re-trigger its descendants. But if possible we think we might skip this issue at the first version to reduce the implementation complexity since it should not affect the correctness. We could considering support it in the following versions.

    3. We would have to add a field isFinished  to OperatorState so that we could not re-run finished sources after failover. However, this would require a new version of checkpoint meta. Currently Flink have an abstract MetaV2V3SerializerBase and have V2 and V3 extends it to share some implementation. To add V4 which is only different from V3 for one field, the current PoC want to introduce a new MetaV3V4SerializerBase extends MetaV2V3SerializerBase to share implementation between V3 and V4. This might looks a little complex and we might need a general mechanism to extend checkpoint meta format. 

    4. With the change StreamTask would have two types of subclasses according to how to implement triggerCheckpoint, one is source tasks that perform checkpoints immediately and another is the non-source tasks that would notify CheckpointBarrierHandler in some way. However, since we have multiple source tasks (legacy and new source) and multiple non-source tasks (one-input, two-input, multiple-input), it would cause the cases that multiple subclasses share the same implementation and  cause code repetition. Currently the PoC introduces a new level of abstraction, namely SourceStreamTasks and NonSourceStreamTasks, but what makes it more complicated is that StreamingIterationHead extends OneInputStreamTask but it need to perform checkpoint as source tasks.

Glad to hear your opinions!

Best,
 Yun

[1] https://github.com/gaoyunhaii/flink/commits/try_checkpoint_6 , starts from commit f8005be1ab5e5124e981e56db7bdf2908f4a969a.

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

Posted by Yun Gao <yu...@aliyun.com>.
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 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 interference (which seems to be a bit not trackable). We could do so by inserting suitable barriers for input channels received but not processed EndOfPartition. For example, if a task with four inputs has received barrier 2 from two input channels, but the other two inputs do not received barrier 2  before EndOfPartition due to the precedent tasks are finished, we could then insert barrier 2 for the last two channels so that we could still finish the checkpoint 2.
    2. As we have discussed, if a tasks finished during we triggering the tasks, it would cause checkpoint failure and we should re-trigger its descendants. But if possible we think we might skip this issue at the first version to reduce the implementation complexity since it should not affect the correctness. We could considering support it in the following versions.

    3. We would have to add a field isFinished  to OperatorState so that we could not re-run finished sources after failover. However, this would require a new version of checkpoint meta. Currently Flink have an abstract MetaV2V3SerializerBase and have V2 and V3 extends it to share some implementation. To add V4 which is only different from V3 for one field, the current PoC want to introduce a new MetaV3V4SerializerBase extends MetaV2V3SerializerBase to share implementation between V3 and V4. This might looks a little complex and we might need a general mechanism to extend checkpoint meta format. 

    4. With the change StreamTask would have two types of subclasses according to how to implement triggerCheckpoint, one is source tasks that perform checkpoints immediately and another is the non-source tasks that would notify CheckpointBarrierHandler in some way. However, since we have multiple source tasks (legacy and new source) and multiple non-source tasks (one-input, two-input, multiple-input), it would cause the cases that multiple subclasses share the same implementation and  cause code repetition. Currently the PoC introduces a new level of abstraction, namely SourceStreamTasks and NonSourceStreamTasks, but what makes it more complicated is that StreamingIterationHead extends OneInputStreamTask but it need to perform checkpoint as source tasks.

Glad to hear your opinions!

Best,
 Yun

[1] https://github.com/gaoyunhaii/flink/commits/try_checkpoint_6 , starts from commit f8005be1ab5e5124e981e56db7bdf2908f4a969a.

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

Posted by Yun Gao <yu...@aliyun.com>.
Hi Till,
Very thanks for the feedbacks !
> 1) When restarting all tasks independent of the status at checkpoint time (finished, running, scheduled), we might allocate more resources than we actually need to run the remaining job. From a scheduling perspective it would be easier if we already know that certain subtasks don't need to be rescheduled. I believe this can be an optimization, though.
> 2) In the section Compatibility, Deprecation and Migration Plan you mentioned that you want to record operators in the CompletedCheckpoint which are fully finished. How will this information be used for constructing a recovered ExecutionGraph? Why wouldn't the same principle work for the task level?

I think the first two issues should be related. The main reason that with external checkpoints the checkpoint might taken from one job and used in another jobs, but we do not have a unique ID to match tasks across jobs. Furthermore, users may also change the parallelism of JobVertex, or even modify the graph structures by adding/removing operators or changing the chain relationship between operators. 
On the other side, currently Flink already provides custom UID for operators, which makes the operators a stable unit for recovery. The current checkpoints are also organized in the unit of operators to support rescale and job Upgrading. 
When restarting from a checkpoint with finished operators, we could only starts the tasks with operators that are not fully finished (namely some subtasks are still running when taking checkpoints). Then during the execution of a single task, we only initialize/open/run/close the operators not fully finished. The Scheduler should be able to compute if a tasks contains not fully finished operators with the current JobGraph and the operator finish states restored from the checkpoints.

> 3) How will checkpointing work together with fully bounded jobs and FLIP-1 (fine grained recovery)?
Currently I think it should be compatible with fully bounded jobs and FLIP-1 since it could be viewed as a completion of the current checkpoint mechanism. Concretely
1. The batch job (with blocking execution mode) should be not affected since checkpoints are not enabled in this case.
2. The bounded job running with pipeline mode would be also supported with checkpoints during it is finishing with the modification. As discussed in the FLIP it should not affect the current behavior after restored for almost all the jobs.
3. The region failover and more fine-grained tasks should also not be affected: similar to the previous behavior, after failover, the failover policy (full/region/fine-grained) decides which tasks to restart and the checkpoint only decides what state are restored for these tasks. The only difference with this modification is that these tasks are now might restored from a checkpoints taken after some tasks are finished. Since the perviously finished tasks would always be skipped by not started or run an empty execution, and the behavior of the previously running tasks should keeps unchanged, the overall behavior should be not affected.


Best,
Yun


------------------------------------------------------------------
From:Till Rohrmann <tr...@apache.org>
Send Time:2020 Oct. 13 (Tue.) 17:25
To:Yun Gao <yu...@aliyun.com>
Cc:Arvid Heise <ar...@ververica.com>; Flink Dev <de...@flink.apache.org>; User-Flink <us...@flink.apache.org>
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 checkpoint time (finished, running, scheduled), we might allocate more resources than we actually need to run the remaining job. From a scheduling perspective it would be easier if we already know that certain subtasks don't need to be rescheduled. I believe this can be an optimization, though.

2) In the section Compatibility, Deprecation and Migration Plan you mentioned that you want to record operators in the CompletedCheckpoint which are fully finished. How will this information be used for constructing a recovered ExecutionGraph? Why wouldn't the same principle work for the task level?

3) How will checkpointing work together with fully bounded jobs and FLIP-1 (fine grained recovery)?

Cheers,
Till
On Tue, Oct 13, 2020 at 9:30 AM Yun Gao <yu...@aliyun.com> wrote:

Hi Arvid,
Very thanks for the comments!
>>> 4) Yes, the interaction is not trivial and also I have not completely thought it through. But in general, I'm currently at the point where I think that we also need non-checkpoint related events in unaligned checkpoints. So just keep that in mind, that we might converge anyhow at this point.
I also agree with that it would be better to keep the unaligned checkpoints behavior on EndOfPartition, I will then double check on this issue again. 

>>> In general, what is helping in this case is to remember that there no unaligned checkpoint barrier ever going to overtake EndOfPartition. So, we can completely ignore the problem on how to store and restore output buffers of a completed task (also important for the next point).
Exactly, we should not need to persist the output buffers for the completed tasks, and that would simply the implementation a lot.

>>> 5) I think we are on the same page and I completely agree that for the MVP/first version, it's completely fine to start and immediately stop. A tad better would be even to not even start the procession loop. 
I also agree with this part. We would keep optimizing the implementation after the first version. 

Best,
Yun   


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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi Till,
Very thanks for the feedbacks !
> 1) When restarting all tasks independent of the status at checkpoint time (finished, running, scheduled), we might allocate more resources than we actually need to run the remaining job. From a scheduling perspective it would be easier if we already know that certain subtasks don't need to be rescheduled. I believe this can be an optimization, though.
> 2) In the section Compatibility, Deprecation and Migration Plan you mentioned that you want to record operators in the CompletedCheckpoint which are fully finished. How will this information be used for constructing a recovered ExecutionGraph? Why wouldn't the same principle work for the task level?

I think the first two issues should be related. The main reason that with external checkpoints the checkpoint might taken from one job and used in another jobs, but we do not have a unique ID to match tasks across jobs. Furthermore, users may also change the parallelism of JobVertex, or even modify the graph structures by adding/removing operators or changing the chain relationship between operators. 
On the other side, currently Flink already provides custom UID for operators, which makes the operators a stable unit for recovery. The current checkpoints are also organized in the unit of operators to support rescale and job Upgrading. 
When restarting from a checkpoint with finished operators, we could only starts the tasks with operators that are not fully finished (namely some subtasks are still running when taking checkpoints). Then during the execution of a single task, we only initialize/open/run/close the operators not fully finished. The Scheduler should be able to compute if a tasks contains not fully finished operators with the current JobGraph and the operator finish states restored from the checkpoints.

> 3) How will checkpointing work together with fully bounded jobs and FLIP-1 (fine grained recovery)?
Currently I think it should be compatible with fully bounded jobs and FLIP-1 since it could be viewed as a completion of the current checkpoint mechanism. Concretely
1. The batch job (with blocking execution mode) should be not affected since checkpoints are not enabled in this case.
2. The bounded job running with pipeline mode would be also supported with checkpoints during it is finishing with the modification. As discussed in the FLIP it should not affect the current behavior after restored for almost all the jobs.
3. The region failover and more fine-grained tasks should also not be affected: similar to the previous behavior, after failover, the failover policy (full/region/fine-grained) decides which tasks to restart and the checkpoint only decides what state are restored for these tasks. The only difference with this modification is that these tasks are now might restored from a checkpoints taken after some tasks are finished. Since the perviously finished tasks would always be skipped by not started or run an empty execution, and the behavior of the previously running tasks should keeps unchanged, the overall behavior should be not affected.


Best,
Yun


------------------------------------------------------------------
From:Till Rohrmann <tr...@apache.org>
Send Time:2020 Oct. 13 (Tue.) 17:25
To:Yun Gao <yu...@aliyun.com>
Cc:Arvid Heise <ar...@ververica.com>; Flink Dev <de...@flink.apache.org>; User-Flink <us...@flink.apache.org>
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 checkpoint time (finished, running, scheduled), we might allocate more resources than we actually need to run the remaining job. From a scheduling perspective it would be easier if we already know that certain subtasks don't need to be rescheduled. I believe this can be an optimization, though.

2) In the section Compatibility, Deprecation and Migration Plan you mentioned that you want to record operators in the CompletedCheckpoint which are fully finished. How will this information be used for constructing a recovered ExecutionGraph? Why wouldn't the same principle work for the task level?

3) How will checkpointing work together with fully bounded jobs and FLIP-1 (fine grained recovery)?

Cheers,
Till
On Tue, Oct 13, 2020 at 9:30 AM Yun Gao <yu...@aliyun.com> wrote:

Hi Arvid,
Very thanks for the comments!
>>> 4) Yes, the interaction is not trivial and also I have not completely thought it through. But in general, I'm currently at the point where I think that we also need non-checkpoint related events in unaligned checkpoints. So just keep that in mind, that we might converge anyhow at this point.
I also agree with that it would be better to keep the unaligned checkpoints behavior on EndOfPartition, I will then double check on this issue again. 

>>> In general, what is helping in this case is to remember that there no unaligned checkpoint barrier ever going to overtake EndOfPartition. So, we can completely ignore the problem on how to store and restore output buffers of a completed task (also important for the next point).
Exactly, we should not need to persist the output buffers for the completed tasks, and that would simply the implementation a lot.

>>> 5) I think we are on the same page and I completely agree that for the MVP/first version, it's completely fine to start and immediately stop. A tad better would be even to not even start the procession loop. 
I also agree with this part. We would keep optimizing the implementation after the first version. 

Best,
Yun   


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

Posted by Till Rohrmann <tr...@apache.org>.
Thanks for starting this discussion Yun Gao,

I have three comments/questions:

1) When restarting all tasks independent of the status at checkpoint time
(finished, running, scheduled), we might allocate more resources than we
actually need to run the remaining job. From a scheduling perspective it
would be easier if we already know that certain subtasks don't need to be
rescheduled. I believe this can be an optimization, though.

2) In the section Compatibility, Deprecation and Migration Plan you
mentioned that you want to record operators in the CompletedCheckpoint
which are fully finished. How will this information be used for
constructing a recovered ExecutionGraph? Why wouldn't the same principle
work for the task level?

3) How will checkpointing work together with fully bounded jobs and FLIP-1
(fine grained recovery)?

Cheers,
Till

On Tue, Oct 13, 2020 at 9:30 AM Yun Gao <yu...@aliyun.com> wrote:

> Hi Arvid,
>
> Very thanks for the comments!
>
> >>> 4) Yes, the interaction is not trivial and also I have not completely
> thought it through. But in general, I'm currently at the point where I
> think that we also need non-checkpoint related events in unaligned
> checkpoints. So just keep that in mind, that we might converge anyhow at
> this point.
>
> I also agree with that it would be better to keep the unaligned
> checkpoints behavior on EndOfPartition, I will then double check on this
> issue again.
>
> >>> In general, what is helping in this case is to remember that there no
> unaligned checkpoint barrier ever going to overtake EndOfPartition. So, we
> can completely ignore the problem on how to store and restore output
> buffers of a completed task (also important for the next point).
>
> Exactly, we should not need to persist the output buffers for the
> completed tasks, and that would simply the implementation a lot.
>
>
> >>> 5) I think we are on the same page and I completely agree that for
> the MVP/first version, it's completely fine to start and immediately stop.
> A tad better would be even to not even start the procession loop.
>
> I also agree with this part. We would keep optimizing the implementation
> after the first version.
>
>
> Best,
>
> Yun
>
>
>
> ------------------------------------------------------------------
> From:Arvid Heise <ar...@ververica.com>
> Send Time:2020 Oct. 13 (Tue.) 03:39
> To:Yun Gao <yu...@aliyun.com>
> Cc:Flink Dev <de...@flink.apache.org>; User-Flink <us...@flink.apache.org>
> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
> Hi Yun,
>
> 4) Yes, the interaction is not trivial and also I have not completely
> thought it through. But in general, I'm currently at the point where I
> think that we also need non-checkpoint related events in unaligned
> checkpoints. So just keep that in mind, that we might converge anyhow at
> this point.
>
> In general, what is helping in this case is to remember that there no
> unaligned checkpoint barrier ever going to overtake EndOfPartition. So, we
> can completely ignore the problem on how to store and restore output
> buffers of a completed task (also important for the next point).
>
> 5) I think we are on the same page and I completely agree that for the
> MVP/first version, it's completely fine to start and immediately stop. A
> tad better would be even to not even start the procession loop.
>
> On Mon, Oct 12, 2020 at 6:18 PM Yun Gao <yu...@aliyun.com> wrote:
>
> Hi Arvid,
>
> Very thanks for the insightful comments! I added the responses for this
> issue under the quota:
>
> >> 1) You call the tasks that get the barriers injected leaf nodes, which
> would make the > sinks the root nodes. That is very similar to how graphs
> in relational algebra are labeled. However, I got the feeling that in
> Flink, we rather iterate from sources to sink, making the sources root
> nodes and the sinks the leaf nodes. However, I have no clue how it's done
> in similar cases, so please take that hint cautiously.
>
> >> 2) I'd make the algorithm to find the subtasks iterative and react in
> CheckpointCoordinator. Let's assume that we inject the barrier at all root
> subtasks (initially all sources). So in the iterative algorithm, whenever
> root A finishes, it looks at all connected subtasks B if they have any
> upstream task left. If not B becomes a new root. That would require to only
> touch a part of the job graph, but would require some callback from
> JobManager to CheckpointCoordinator.
>
>
> I think I should have used a bad name of "leaf nodes", in fact I think we
> should have the same thoughts that we start with the source nodes to find
> all the nodes whose precedent nodes are all finished. It would be much
> better to call these nodes (which we would trigger) as "root nodes". I'll
> modify the FLIP to change the names to "root nodes".
>
> >> 2b) We also need to be careful for out-of-sync updates: if the root is
> about to finish, we could send the barrier to it from
> CheckpointCoordinator, but at the time it arrives, the subtask is finished
> already.
>
> Exactly. When the checkpoint triggers a task but found the task is not
> there, it may then further check if the task has been finished, if so, it
> should then re-check its descendants to see if there are new "root nodes"
> to trigger.
>
> >> 3) An implied change is that checkpoints are not aborted anymore at
> EndOfPartition, which is good, but might be explicitly added.
>
> Yes, currently barrier alignment would fail the current checkpoint on
> EndOfPartition, and we would modify the behavior.
>
> >> 4) The interaction between unaligned checkpoint and EndOfPartition is a
> bit ambiguous: What happens when an unaligned checkpoint is started and
> then one input channel contains the EndOfPartition event? From the written
> description, it sounds to me like, we move back to an aligned checkpoint
> for the whole receiving task. However, that is neither easily possible nor
> necessary. Imho it would be enough to also store the EndOfPartition in the
> channel state.
>
>
> Very thanks for the suggestions on this issue and in fact I did stuck on
> it for some time. Previously for me one implementation detail issue is that
> EndOfPartition seems not be able to overtake the previous buffers easily as
> CheckpointBarrier does, otherwise it might start destroying the input
> channels if all EndOfPartitions are received.
>
> Therefore, although we could also persistent the channels with
> EndOfPartition:
>
> 1. Start persisting the channels when CheckpointUnaligner received barrier
> (if not all precendant tasks are finished) or received triggering (if all
> precendant tasks are finished).
>
> 2. The persisting actually stops when onBuffer received EndOfPartition.
>
> After the last channel stopped persisting, CheckpointUnaligner still need
> to wait till all the previous buffers are processed before complete the
> allBarriersReceivedFuture. Therefore it would not be able to accelerate the
> checkpoint in this case.
>
> After some rethinking today currently I think we might inserts some
> additional virtual events into receivedBuffer when received EndOfPartition
> and allows these virtual events to overtake the previous buffers. I'll try
> to double check if it is feasible and let me know if there are also other
> solutions on this issue :).
>
> > 5) I'd expand the recovery section a bit. It would be the first time
> that we recover an incomplete DAG. Afaik the subtasks are deployed before
> the state is recovered, so at some point, the subtasks either need to be
> removed again or maybe we could even avoid them being created in the first
> place.
>
> I also agree that finally we should not "restarted" the finished tasks in
> some way. It seems not start it in the first place would be better. We
> should be able to bookkeep additional information in the checkpoint meta
> about which operators are fully finished, and the scheduler could restore
> 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 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 solution gradually in the next steps.
>
>
> Best,
>
> Yun
>
> ------------------------------------------------------------------
> From:Arvid Heise <ar...@ververica.com>
> Send Time:2020 Oct. 12 (Mon.) 15:38
> To:Yun Gao <yu...@aliyun.com>
> Cc:Flink Dev <de...@flink.apache.org>; User-Flink <us...@flink.apache.org>
> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
> Hi Yun,
>
> Thank you for starting the discussion. This will solve one of the
> long-standing issues [1] that confuse users. I'm also a big fan of option
> 3. It is also a bit closer to Chandy-Lamport again.
>
> A couple of comments:
>
> 1) You call the tasks that get the barriers injected leaf nodes, which
> would make the sinks the root nodes. That is very similar to how graphs in
> relational algebra are labeled. However, I got the feeling that in Flink,
> we rather iterate from sources to sink, making the sources root nodes and
> the sinks the leaf nodes. However, I have no clue how it's done in similar
> cases, so please take that hint cautiously.
> 2) I'd make the algorithm to find the subtasks iterative and react in
> CheckpointCoordinator. Let's assume that we inject the barrier at all root
> subtasks (initially all sources). So in the iterative algorithm, whenever
> root A finishes, it looks at all connected subtasks B if they have any
> upstream task left. If not B becomes a new root. That would require to only
> touch a part of the job graph, but would require some callback from
> JobManager to CheckpointCoordinator.
> 2b) We also need to be careful for out-of-sync updates: if the root is
> about to finish, we could send the barrier to it from
> CheckpointCoordinator, but at the time it arrives, the subtask is finished
> already.
> 3) An implied change is that checkpoints are not aborted anymore at EndOfPartition,
> which is good, but might be explicitly added.
> 4) The interaction between unaligned checkpoint and EndOfPartition is a
> bit ambiguous: What happens when an unaligned checkpoint is started and
> then one input channel contains the EndOfPartition event? From the
> written description, it sounds to me like, we move back to an aligned
> checkpoint for the whole receiving task. However, that is neither easily
> possible nor necessary. Imho it would be enough to also store the EndOfPartition
> in the channel state.
> 5) I'd expand the recovery section a bit. It would be the first time that
> we recover an incomplete DAG. Afaik the subtasks are deployed before the
> state is recovered, so at some point, the subtasks either need to be
> removed again or maybe we could even avoid them being created in the first
> place.
>
> [1] https://issues.apache.org/jira/browse/FLINK-2491
>
> On Fri, Oct 9, 2020 at 8:22 AM Yun Gao <yu...@aliyun.com> wrote:
> Hi, devs & users
>
> Very sorry for the spoiled formats, I resent the discussion as follows.
>
>
> 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 tasks finished, which causes some problems for bounded or mixed jobs:
>         1.
> Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5].
>         2.
> The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
>
>
> Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6].
>
> Best,
> Yun
>
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> [2]
> https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
> [3]
> https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
> [4]
> https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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 <yu...@aliyun.com.INVALID>
> *Send Date:*Fri Oct 9 14:16:52 2020
> *Recipients:*Flink Dev <de...@flink.apache.org>, User-Flink <
> user@flink.apache.org>
> *Subject:*[DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
> 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 tasks finished, which causes some problems for bounded or mixed jobs:
>
> Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5].
>
> The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
>
> Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6].
> Best,
> Yun
>
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> [2]
> https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
> [3]
> https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
> [4]
> https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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
>
>
> --
>
> Arvid Heise| Senior Java Developer
>
> <https://www.ververica.com/>
>
> Follow us @VervericaData
>
> --
>
> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
> Conference
>
> Stream Processing | Event Driven | Real Time
>
> --
>
> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>
> --
> Ververica GmbH
> Registered at Amtsgericht Charlottenburg: HRB 158244 B
> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
> (Toni) Cheng
>
>
>
> --
>
> Arvid Heise | Senior Java Developer
>
> <https://www.ververica.com/>
>
> Follow us @VervericaData
>
> --
>
> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
> Conference
>
> Stream Processing | Event Driven | Real Time
>
> --
>
> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>
> --
> Ververica GmbH
> Registered at Amtsgericht Charlottenburg: HRB 158244 B
> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
> (Toni) Cheng
>
>
>

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

Posted by Till Rohrmann <tr...@apache.org>.
Thanks for starting this discussion Yun Gao,

I have three comments/questions:

1) When restarting all tasks independent of the status at checkpoint time
(finished, running, scheduled), we might allocate more resources than we
actually need to run the remaining job. From a scheduling perspective it
would be easier if we already know that certain subtasks don't need to be
rescheduled. I believe this can be an optimization, though.

2) In the section Compatibility, Deprecation and Migration Plan you
mentioned that you want to record operators in the CompletedCheckpoint
which are fully finished. How will this information be used for
constructing a recovered ExecutionGraph? Why wouldn't the same principle
work for the task level?

3) How will checkpointing work together with fully bounded jobs and FLIP-1
(fine grained recovery)?

Cheers,
Till

On Tue, Oct 13, 2020 at 9:30 AM Yun Gao <yu...@aliyun.com> wrote:

> Hi Arvid,
>
> Very thanks for the comments!
>
> >>> 4) Yes, the interaction is not trivial and also I have not completely
> thought it through. But in general, I'm currently at the point where I
> think that we also need non-checkpoint related events in unaligned
> checkpoints. So just keep that in mind, that we might converge anyhow at
> this point.
>
> I also agree with that it would be better to keep the unaligned
> checkpoints behavior on EndOfPartition, I will then double check on this
> issue again.
>
> >>> In general, what is helping in this case is to remember that there no
> unaligned checkpoint barrier ever going to overtake EndOfPartition. So, we
> can completely ignore the problem on how to store and restore output
> buffers of a completed task (also important for the next point).
>
> Exactly, we should not need to persist the output buffers for the
> completed tasks, and that would simply the implementation a lot.
>
>
> >>> 5) I think we are on the same page and I completely agree that for
> the MVP/first version, it's completely fine to start and immediately stop.
> A tad better would be even to not even start the procession loop.
>
> I also agree with this part. We would keep optimizing the implementation
> after the first version.
>
>
> Best,
>
> Yun
>
>
>
> ------------------------------------------------------------------
> From:Arvid Heise <ar...@ververica.com>
> Send Time:2020 Oct. 13 (Tue.) 03:39
> To:Yun Gao <yu...@aliyun.com>
> Cc:Flink Dev <de...@flink.apache.org>; User-Flink <us...@flink.apache.org>
> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
> Hi Yun,
>
> 4) Yes, the interaction is not trivial and also I have not completely
> thought it through. But in general, I'm currently at the point where I
> think that we also need non-checkpoint related events in unaligned
> checkpoints. So just keep that in mind, that we might converge anyhow at
> this point.
>
> In general, what is helping in this case is to remember that there no
> unaligned checkpoint barrier ever going to overtake EndOfPartition. So, we
> can completely ignore the problem on how to store and restore output
> buffers of a completed task (also important for the next point).
>
> 5) I think we are on the same page and I completely agree that for the
> MVP/first version, it's completely fine to start and immediately stop. A
> tad better would be even to not even start the procession loop.
>
> On Mon, Oct 12, 2020 at 6:18 PM Yun Gao <yu...@aliyun.com> wrote:
>
> Hi Arvid,
>
> Very thanks for the insightful comments! I added the responses for this
> issue under the quota:
>
> >> 1) You call the tasks that get the barriers injected leaf nodes, which
> would make the > sinks the root nodes. That is very similar to how graphs
> in relational algebra are labeled. However, I got the feeling that in
> Flink, we rather iterate from sources to sink, making the sources root
> nodes and the sinks the leaf nodes. However, I have no clue how it's done
> in similar cases, so please take that hint cautiously.
>
> >> 2) I'd make the algorithm to find the subtasks iterative and react in
> CheckpointCoordinator. Let's assume that we inject the barrier at all root
> subtasks (initially all sources). So in the iterative algorithm, whenever
> root A finishes, it looks at all connected subtasks B if they have any
> upstream task left. If not B becomes a new root. That would require to only
> touch a part of the job graph, but would require some callback from
> JobManager to CheckpointCoordinator.
>
>
> I think I should have used a bad name of "leaf nodes", in fact I think we
> should have the same thoughts that we start with the source nodes to find
> all the nodes whose precedent nodes are all finished. It would be much
> better to call these nodes (which we would trigger) as "root nodes". I'll
> modify the FLIP to change the names to "root nodes".
>
> >> 2b) We also need to be careful for out-of-sync updates: if the root is
> about to finish, we could send the barrier to it from
> CheckpointCoordinator, but at the time it arrives, the subtask is finished
> already.
>
> Exactly. When the checkpoint triggers a task but found the task is not
> there, it may then further check if the task has been finished, if so, it
> should then re-check its descendants to see if there are new "root nodes"
> to trigger.
>
> >> 3) An implied change is that checkpoints are not aborted anymore at
> EndOfPartition, which is good, but might be explicitly added.
>
> Yes, currently barrier alignment would fail the current checkpoint on
> EndOfPartition, and we would modify the behavior.
>
> >> 4) The interaction between unaligned checkpoint and EndOfPartition is a
> bit ambiguous: What happens when an unaligned checkpoint is started and
> then one input channel contains the EndOfPartition event? From the written
> description, it sounds to me like, we move back to an aligned checkpoint
> for the whole receiving task. However, that is neither easily possible nor
> necessary. Imho it would be enough to also store the EndOfPartition in the
> channel state.
>
>
> Very thanks for the suggestions on this issue and in fact I did stuck on
> it for some time. Previously for me one implementation detail issue is that
> EndOfPartition seems not be able to overtake the previous buffers easily as
> CheckpointBarrier does, otherwise it might start destroying the input
> channels if all EndOfPartitions are received.
>
> Therefore, although we could also persistent the channels with
> EndOfPartition:
>
> 1. Start persisting the channels when CheckpointUnaligner received barrier
> (if not all precendant tasks are finished) or received triggering (if all
> precendant tasks are finished).
>
> 2. The persisting actually stops when onBuffer received EndOfPartition.
>
> After the last channel stopped persisting, CheckpointUnaligner still need
> to wait till all the previous buffers are processed before complete the
> allBarriersReceivedFuture. Therefore it would not be able to accelerate the
> checkpoint in this case.
>
> After some rethinking today currently I think we might inserts some
> additional virtual events into receivedBuffer when received EndOfPartition
> and allows these virtual events to overtake the previous buffers. I'll try
> to double check if it is feasible and let me know if there are also other
> solutions on this issue :).
>
> > 5) I'd expand the recovery section a bit. It would be the first time
> that we recover an incomplete DAG. Afaik the subtasks are deployed before
> the state is recovered, so at some point, the subtasks either need to be
> removed again or maybe we could even avoid them being created in the first
> place.
>
> I also agree that finally we should not "restarted" the finished tasks in
> some way. It seems not start it in the first place would be better. We
> should be able to bookkeep additional information in the checkpoint meta
> about which operators are fully finished, and the scheduler could restore
> 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 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 solution gradually in the next steps.
>
>
> Best,
>
> Yun
>
> ------------------------------------------------------------------
> From:Arvid Heise <ar...@ververica.com>
> Send Time:2020 Oct. 12 (Mon.) 15:38
> To:Yun Gao <yu...@aliyun.com>
> Cc:Flink Dev <de...@flink.apache.org>; User-Flink <us...@flink.apache.org>
> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
> Hi Yun,
>
> Thank you for starting the discussion. This will solve one of the
> long-standing issues [1] that confuse users. I'm also a big fan of option
> 3. It is also a bit closer to Chandy-Lamport again.
>
> A couple of comments:
>
> 1) You call the tasks that get the barriers injected leaf nodes, which
> would make the sinks the root nodes. That is very similar to how graphs in
> relational algebra are labeled. However, I got the feeling that in Flink,
> we rather iterate from sources to sink, making the sources root nodes and
> the sinks the leaf nodes. However, I have no clue how it's done in similar
> cases, so please take that hint cautiously.
> 2) I'd make the algorithm to find the subtasks iterative and react in
> CheckpointCoordinator. Let's assume that we inject the barrier at all root
> subtasks (initially all sources). So in the iterative algorithm, whenever
> root A finishes, it looks at all connected subtasks B if they have any
> upstream task left. If not B becomes a new root. That would require to only
> touch a part of the job graph, but would require some callback from
> JobManager to CheckpointCoordinator.
> 2b) We also need to be careful for out-of-sync updates: if the root is
> about to finish, we could send the barrier to it from
> CheckpointCoordinator, but at the time it arrives, the subtask is finished
> already.
> 3) An implied change is that checkpoints are not aborted anymore at EndOfPartition,
> which is good, but might be explicitly added.
> 4) The interaction between unaligned checkpoint and EndOfPartition is a
> bit ambiguous: What happens when an unaligned checkpoint is started and
> then one input channel contains the EndOfPartition event? From the
> written description, it sounds to me like, we move back to an aligned
> checkpoint for the whole receiving task. However, that is neither easily
> possible nor necessary. Imho it would be enough to also store the EndOfPartition
> in the channel state.
> 5) I'd expand the recovery section a bit. It would be the first time that
> we recover an incomplete DAG. Afaik the subtasks are deployed before the
> state is recovered, so at some point, the subtasks either need to be
> removed again or maybe we could even avoid them being created in the first
> place.
>
> [1] https://issues.apache.org/jira/browse/FLINK-2491
>
> On Fri, Oct 9, 2020 at 8:22 AM Yun Gao <yu...@aliyun.com> wrote:
> Hi, devs & users
>
> Very sorry for the spoiled formats, I resent the discussion as follows.
>
>
> 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 tasks finished, which causes some problems for bounded or mixed jobs:
>         1.
> Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5].
>         2.
> The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
>
>
> Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6].
>
> Best,
> Yun
>
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> [2]
> https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
> [3]
> https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
> [4]
> https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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 <yu...@aliyun.com.INVALID>
> *Send Date:*Fri Oct 9 14:16:52 2020
> *Recipients:*Flink Dev <de...@flink.apache.org>, User-Flink <
> user@flink.apache.org>
> *Subject:*[DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
> 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 tasks finished, which causes some problems for bounded or mixed jobs:
>
> Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5].
>
> The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
>
> Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6].
> Best,
> Yun
>
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> [2]
> https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
> [3]
> https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
> [4]
> https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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
>
>
> --
>
> Arvid Heise| Senior Java Developer
>
> <https://www.ververica.com/>
>
> Follow us @VervericaData
>
> --
>
> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
> Conference
>
> Stream Processing | Event Driven | Real Time
>
> --
>
> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>
> --
> Ververica GmbH
> Registered at Amtsgericht Charlottenburg: HRB 158244 B
> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
> (Toni) Cheng
>
>
>
> --
>
> Arvid Heise | Senior Java Developer
>
> <https://www.ververica.com/>
>
> Follow us @VervericaData
>
> --
>
> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
> Conference
>
> Stream Processing | Event Driven | Real Time
>
> --
>
> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>
> --
> Ververica GmbH
> Registered at Amtsgericht Charlottenburg: HRB 158244 B
> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
> (Toni) Cheng
>
>
>

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi Arvid,
Very thanks for the comments!
>>> 4) Yes, the interaction is not trivial and also I have not completely thought it through. But in general, I'm currently at the point where I think that we also need non-checkpoint related events in unaligned checkpoints. So just keep that in mind, that we might converge anyhow at this point.
I also agree with that it would be better to keep the unaligned checkpoints behavior on EndOfPartition, I will then double check on this issue again. 

>>> In general, what is helping in this case is to remember that there no unaligned checkpoint barrier ever going to overtake EndOfPartition. So, we can completely ignore the problem on how to store and restore output buffers of a completed task (also important for the next point).
Exactly, we should not need to persist the output buffers for the completed tasks, and that would simply the implementation a lot.

>>> 5) I think we are on the same page and I completely agree that for the MVP/first version, it's completely fine to start and immediately stop. A tad better would be even to not even start the procession loop. 
I also agree with this part. We would keep optimizing the implementation after the first version. 

Best,
Yun   




------------------------------------------------------------------
From:Arvid Heise <ar...@ververica.com>
Send Time:2020 Oct. 13 (Tue.) 03:39
To:Yun Gao <yu...@aliyun.com>
Cc:Flink Dev <de...@flink.apache.org>; User-Flink <us...@flink.apache.org>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi Yun,

4) Yes, the interaction is not trivial and also I have not completely thought it through. But in general, I'm currently at the point where I think that we also need non-checkpoint related events in unaligned checkpoints. So just keep that in mind, that we might converge anyhow at this point.

In general, what is helping in this case is to remember that there no unaligned checkpoint barrier ever going to overtake EndOfPartition. So, we can completely ignore the problem on how to store and restore output buffers of a completed task (also important for the next point).

5) I think we are on the same page and I completely agree that for the MVP/first version, it's completely fine to start and immediately stop. A tad better would be even to not even start the procession loop. 

On Mon, Oct 12, 2020 at 6:18 PM Yun Gao <yu...@aliyun.com> wrote:

Hi Arvid,
Very thanks for the insightful comments! I added the responses for this issue under the quota: 
>> 1) You call the tasks that get the barriers injected leaf nodes, which would make the > sinks the root nodes. That is very similar to how graphs in relational algebra are labeled. However, I got the feeling that in Flink, we rather iterate from sources to sink, making the sources root nodes and the sinks the leaf nodes. However, I have no clue how it's done in similar cases, so please take that hint cautiously.
>> 2) I'd make the algorithm to find the subtasks iterative and react in CheckpointCoordinator. Let's assume that we inject the barrier at all root subtasks (initially all sources). So in the iterative algorithm, whenever root A finishes, it looks at all connected subtasks B if they have any upstream task left. If not B becomes a new root. That would require to only touch a part of the job graph, but would require some callback from JobManager to CheckpointCoordinator.

I think I should have used a bad name of "leaf nodes", in fact I think we should have the same thoughts that we start with the source nodes to find all the nodes whose precedent nodes are all finished. It would be much better to call these nodes (which we would trigger) as "root nodes". I'll modify the FLIP to change the names to "root nodes".
>> 2b) We also need to be careful for out-of-sync updates: if the root is about to finish, we could send the barrier to it from CheckpointCoordinator, but at the time it arrives, the subtask is finished already.
Exactly. When the checkpoint triggers a task but found the task is not there, it may then further check if the task has been finished, if so, it should then re-check its descendants to see if there are new "root nodes" to trigger.
>> 3) An implied change is that checkpoints are not aborted anymore at EndOfPartition, which is good, but might be explicitly added.
Yes, currently barrier alignment would fail the current checkpoint on EndOfPartition, and we would modify the behavior.
>> 4) The interaction between unaligned checkpoint and EndOfPartition is a bit ambiguous: What happens when an unaligned checkpoint is started and then one input channel contains the EndOfPartition event? From the written description, it sounds to me like, we move back to an aligned checkpoint for the whole receiving task. However, that is neither easily possible nor necessary. Imho it would be enough to also store the EndOfPartition in the channel state.

Very thanks for the suggestions on this issue and in fact I did stuck on it for some time. Previously for me one implementation detail issue is that EndOfPartition seems not be able to overtake the previous buffers easily as CheckpointBarrier does, otherwise it might start destroying the input channels if all EndOfPartitions are received.
Therefore, although we could also persistent the channels with EndOfPartition:
1. Start persisting the channels when CheckpointUnaligner received barrier (if not all precendant tasks are finished) or received triggering (if all precendant tasks are finished).
2. The persisting actually stops when onBuffer received EndOfPartition.
After the last channel stopped persisting, CheckpointUnaligner still need to wait till all the previous buffers are processed before complete the allBarriersReceivedFuture. Therefore it would not be able to accelerate the checkpoint in this case.
After some rethinking today currently I think we might inserts some additional virtual events into receivedBuffer when received EndOfPartition and allows these virtual events to overtake the previous buffers. I'll try to double check if it is feasible and let me know if there are also other solutions on this issue :). 
> 5) I'd expand the recovery section a bit. It would be the first time that we recover an incomplete DAG. Afaik the subtasks are deployed before the state is recovered, so at some point, the subtasks either need to be removed again or maybe we could even avoid them being created in the first place.
I also agree that finally we should not "restarted" the finished tasks in some way. It seems not start it in the first place would be better. We should be able to bookkeep additional information in the checkpoint meta about which operators are fully finished, and the scheduler could restore 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 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 solution gradually in the next steps. 

Best,
Yun

------------------------------------------------------------------
From:Arvid Heise <ar...@ververica.com>
Send Time:2020 Oct. 12 (Mon.) 15:38
To:Yun Gao <yu...@aliyun.com>
Cc:Flink Dev <de...@flink.apache.org>; User-Flink <us...@flink.apache.org>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi Yun,

Thank you for starting the discussion. This will solve one of the long-standing issues [1] that confuse users. I'm also a big fan of option 3. It is also a bit closer to Chandy-Lamport again.

A couple of comments:

1) You call the tasks that get the barriers injected leaf nodes, which would make the sinks the root nodes. That is very similar to how graphs in relational algebra are labeled. However, I got the feeling that in Flink, we rather iterate from sources to sink, making the sources root nodes and the sinks the leaf nodes. However, I have no clue how it's done in similar cases, so please take that hint cautiously.
2) I'd make the algorithm to find the subtasks iterative and react in CheckpointCoordinator. Let's assume that we inject the barrier at all root subtasks (initially all sources). So in the iterative algorithm, whenever root A finishes, it looks at all connected subtasks B if they have any upstream task left. If not B becomes a new root. That would require to only touch a part of the job graph, but would require some callback from JobManager to CheckpointCoordinator. 
2b) We also need to be careful for out-of-sync updates: if the root is about to finish, we could send the barrier to it from CheckpointCoordinator, but at the time it arrives, the subtask is finished already.
3) An implied change is that checkpoints are not aborted anymore at EndOfPartition, which is good, but might be explicitly added.
4) The interaction between unaligned checkpoint and EndOfPartition is a bit ambiguous: What happens when an unaligned checkpoint is started and then one input channel contains the EndOfPartition event? From the written description, it sounds to me like, we move back to an aligned checkpoint for the whole receiving task. However, that is neither easily possible nor necessary. Imho it would be enough to also store the EndOfPartition in the channel state.
5) I'd expand the recovery section a bit. It would be the first time that we recover an incomplete DAG. Afaik the subtasks are deployed before the state is recovered, so at some point, the subtasks either need to be removed again or maybe we could even avoid them being created in the first place.

[1] https://issues.apache.org/jira/browse/FLINK-2491
On Fri, Oct 9, 2020 at 8:22 AM Yun Gao <yu...@aliyun.com> wrote:
Hi, devs & users

Very sorry for the spoiled formats, I resent the discussion as follows.

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 tasks finished, which causes some problems for bounded or mixed jobs:
        1. Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5]. 
        2. The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.

Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6]. 

Best,
Yun

[1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
[2] https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
[3] https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
[4] https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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 <yu...@aliyun.com.INVALID>
Send Date:Fri Oct 9 14:16:52 2020
Recipients:Flink Dev <de...@flink.apache.org>, User-Flink <us...@flink.apache.org>
Subject:[DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
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 tasks finished, which causes some problems for bounded or mixed jobs:
Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5]. 
The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6]. 
Best,
Yun

[1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
[2] https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
[3] https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
[4] https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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

-- 
Arvid Heise| Senior Java Developer

Follow us @VervericaData
--
Join Flink Forward - The Apache Flink Conference
Stream Processing | Event Driven | Real Time
--
Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
--Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji (Toni) Cheng



-- 
Arvid Heise | Senior Java Developer

Follow us @VervericaData
--
Join Flink Forward - The Apache Flink Conference
Stream Processing | Event Driven | Real Time
--
Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
--Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji (Toni) Cheng


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

Posted by Yun Gao <yu...@aliyun.com>.
Hi Arvid,
Very thanks for the comments!
>>> 4) Yes, the interaction is not trivial and also I have not completely thought it through. But in general, I'm currently at the point where I think that we also need non-checkpoint related events in unaligned checkpoints. So just keep that in mind, that we might converge anyhow at this point.
I also agree with that it would be better to keep the unaligned checkpoints behavior on EndOfPartition, I will then double check on this issue again. 

>>> In general, what is helping in this case is to remember that there no unaligned checkpoint barrier ever going to overtake EndOfPartition. So, we can completely ignore the problem on how to store and restore output buffers of a completed task (also important for the next point).
Exactly, we should not need to persist the output buffers for the completed tasks, and that would simply the implementation a lot.

>>> 5) I think we are on the same page and I completely agree that for the MVP/first version, it's completely fine to start and immediately stop. A tad better would be even to not even start the procession loop. 
I also agree with this part. We would keep optimizing the implementation after the first version. 

Best,
Yun   




------------------------------------------------------------------
From:Arvid Heise <ar...@ververica.com>
Send Time:2020 Oct. 13 (Tue.) 03:39
To:Yun Gao <yu...@aliyun.com>
Cc:Flink Dev <de...@flink.apache.org>; User-Flink <us...@flink.apache.org>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi Yun,

4) Yes, the interaction is not trivial and also I have not completely thought it through. But in general, I'm currently at the point where I think that we also need non-checkpoint related events in unaligned checkpoints. So just keep that in mind, that we might converge anyhow at this point.

In general, what is helping in this case is to remember that there no unaligned checkpoint barrier ever going to overtake EndOfPartition. So, we can completely ignore the problem on how to store and restore output buffers of a completed task (also important for the next point).

5) I think we are on the same page and I completely agree that for the MVP/first version, it's completely fine to start and immediately stop. A tad better would be even to not even start the procession loop. 

On Mon, Oct 12, 2020 at 6:18 PM Yun Gao <yu...@aliyun.com> wrote:

Hi Arvid,
Very thanks for the insightful comments! I added the responses for this issue under the quota: 
>> 1) You call the tasks that get the barriers injected leaf nodes, which would make the > sinks the root nodes. That is very similar to how graphs in relational algebra are labeled. However, I got the feeling that in Flink, we rather iterate from sources to sink, making the sources root nodes and the sinks the leaf nodes. However, I have no clue how it's done in similar cases, so please take that hint cautiously.
>> 2) I'd make the algorithm to find the subtasks iterative and react in CheckpointCoordinator. Let's assume that we inject the barrier at all root subtasks (initially all sources). So in the iterative algorithm, whenever root A finishes, it looks at all connected subtasks B if they have any upstream task left. If not B becomes a new root. That would require to only touch a part of the job graph, but would require some callback from JobManager to CheckpointCoordinator.

I think I should have used a bad name of "leaf nodes", in fact I think we should have the same thoughts that we start with the source nodes to find all the nodes whose precedent nodes are all finished. It would be much better to call these nodes (which we would trigger) as "root nodes". I'll modify the FLIP to change the names to "root nodes".
>> 2b) We also need to be careful for out-of-sync updates: if the root is about to finish, we could send the barrier to it from CheckpointCoordinator, but at the time it arrives, the subtask is finished already.
Exactly. When the checkpoint triggers a task but found the task is not there, it may then further check if the task has been finished, if so, it should then re-check its descendants to see if there are new "root nodes" to trigger.
>> 3) An implied change is that checkpoints are not aborted anymore at EndOfPartition, which is good, but might be explicitly added.
Yes, currently barrier alignment would fail the current checkpoint on EndOfPartition, and we would modify the behavior.
>> 4) The interaction between unaligned checkpoint and EndOfPartition is a bit ambiguous: What happens when an unaligned checkpoint is started and then one input channel contains the EndOfPartition event? From the written description, it sounds to me like, we move back to an aligned checkpoint for the whole receiving task. However, that is neither easily possible nor necessary. Imho it would be enough to also store the EndOfPartition in the channel state.

Very thanks for the suggestions on this issue and in fact I did stuck on it for some time. Previously for me one implementation detail issue is that EndOfPartition seems not be able to overtake the previous buffers easily as CheckpointBarrier does, otherwise it might start destroying the input channels if all EndOfPartitions are received.
Therefore, although we could also persistent the channels with EndOfPartition:
1. Start persisting the channels when CheckpointUnaligner received barrier (if not all precendant tasks are finished) or received triggering (if all precendant tasks are finished).
2. The persisting actually stops when onBuffer received EndOfPartition.
After the last channel stopped persisting, CheckpointUnaligner still need to wait till all the previous buffers are processed before complete the allBarriersReceivedFuture. Therefore it would not be able to accelerate the checkpoint in this case.
After some rethinking today currently I think we might inserts some additional virtual events into receivedBuffer when received EndOfPartition and allows these virtual events to overtake the previous buffers. I'll try to double check if it is feasible and let me know if there are also other solutions on this issue :). 
> 5) I'd expand the recovery section a bit. It would be the first time that we recover an incomplete DAG. Afaik the subtasks are deployed before the state is recovered, so at some point, the subtasks either need to be removed again or maybe we could even avoid them being created in the first place.
I also agree that finally we should not "restarted" the finished tasks in some way. It seems not start it in the first place would be better. We should be able to bookkeep additional information in the checkpoint meta about which operators are fully finished, and the scheduler could restore 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 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 solution gradually in the next steps. 

Best,
Yun

------------------------------------------------------------------
From:Arvid Heise <ar...@ververica.com>
Send Time:2020 Oct. 12 (Mon.) 15:38
To:Yun Gao <yu...@aliyun.com>
Cc:Flink Dev <de...@flink.apache.org>; User-Flink <us...@flink.apache.org>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi Yun,

Thank you for starting the discussion. This will solve one of the long-standing issues [1] that confuse users. I'm also a big fan of option 3. It is also a bit closer to Chandy-Lamport again.

A couple of comments:

1) You call the tasks that get the barriers injected leaf nodes, which would make the sinks the root nodes. That is very similar to how graphs in relational algebra are labeled. However, I got the feeling that in Flink, we rather iterate from sources to sink, making the sources root nodes and the sinks the leaf nodes. However, I have no clue how it's done in similar cases, so please take that hint cautiously.
2) I'd make the algorithm to find the subtasks iterative and react in CheckpointCoordinator. Let's assume that we inject the barrier at all root subtasks (initially all sources). So in the iterative algorithm, whenever root A finishes, it looks at all connected subtasks B if they have any upstream task left. If not B becomes a new root. That would require to only touch a part of the job graph, but would require some callback from JobManager to CheckpointCoordinator. 
2b) We also need to be careful for out-of-sync updates: if the root is about to finish, we could send the barrier to it from CheckpointCoordinator, but at the time it arrives, the subtask is finished already.
3) An implied change is that checkpoints are not aborted anymore at EndOfPartition, which is good, but might be explicitly added.
4) The interaction between unaligned checkpoint and EndOfPartition is a bit ambiguous: What happens when an unaligned checkpoint is started and then one input channel contains the EndOfPartition event? From the written description, it sounds to me like, we move back to an aligned checkpoint for the whole receiving task. However, that is neither easily possible nor necessary. Imho it would be enough to also store the EndOfPartition in the channel state.
5) I'd expand the recovery section a bit. It would be the first time that we recover an incomplete DAG. Afaik the subtasks are deployed before the state is recovered, so at some point, the subtasks either need to be removed again or maybe we could even avoid them being created in the first place.

[1] https://issues.apache.org/jira/browse/FLINK-2491
On Fri, Oct 9, 2020 at 8:22 AM Yun Gao <yu...@aliyun.com> wrote:
Hi, devs & users

Very sorry for the spoiled formats, I resent the discussion as follows.

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 tasks finished, which causes some problems for bounded or mixed jobs:
        1. Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5]. 
        2. The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.

Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6]. 

Best,
Yun

[1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
[2] https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
[3] https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
[4] https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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 <yu...@aliyun.com.INVALID>
Send Date:Fri Oct 9 14:16:52 2020
Recipients:Flink Dev <de...@flink.apache.org>, User-Flink <us...@flink.apache.org>
Subject:[DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
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 tasks finished, which causes some problems for bounded or mixed jobs:
Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5]. 
The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6]. 
Best,
Yun

[1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
[2] https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
[3] https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
[4] https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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

-- 
Arvid Heise| Senior Java Developer

Follow us @VervericaData
--
Join Flink Forward - The Apache Flink Conference
Stream Processing | Event Driven | Real Time
--
Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
--Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji (Toni) Cheng



-- 
Arvid Heise | Senior Java Developer

Follow us @VervericaData
--
Join Flink Forward - The Apache Flink Conference
Stream Processing | Event Driven | Real Time
--
Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
--Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji (Toni) Cheng


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

Posted by Arvid Heise <ar...@ververica.com>.
Hi Yun,

4) Yes, the interaction is not trivial and also I have not completely
thought it through. But in general, I'm currently at the point where I
think that we also need non-checkpoint related events in unaligned
checkpoints. So just keep that in mind, that we might converge anyhow at
this point.

In general, what is helping in this case is to remember that there no
unaligned checkpoint barrier ever going to overtake EndOfPartition. So, we
can completely ignore the problem on how to store and restore output
buffers of a completed task (also important for the next point).

5) I think we are on the same page and I completely agree that for the
MVP/first version, it's completely fine to start and immediately stop. A
tad better would be even to not even start the procession loop.

On Mon, Oct 12, 2020 at 6:18 PM Yun Gao <yu...@aliyun.com> wrote:

> Hi Arvid,
>
> Very thanks for the insightful comments! I added the responses for this
> issue under the quota:
>
> >> 1) You call the tasks that get the barriers injected leaf nodes, which
> would make the > sinks the root nodes. That is very similar to how graphs
> in relational algebra are labeled. However, I got the feeling that in
> Flink, we rather iterate from sources to sink, making the sources root
> nodes and the sinks the leaf nodes. However, I have no clue how it's done
> in similar cases, so please take that hint cautiously.
>
> >> 2) I'd make the algorithm to find the subtasks iterative and react in
> CheckpointCoordinator. Let's assume that we inject the barrier at all root
> subtasks (initially all sources). So in the iterative algorithm, whenever
> root A finishes, it looks at all connected subtasks B if they have any
> upstream task left. If not B becomes a new root. That would require to only
> touch a part of the job graph, but would require some callback from
> JobManager to CheckpointCoordinator.
>
>
> I think I should have used a bad name of "leaf nodes", in fact I think we
> should have the same thoughts that we start with the source nodes to find
> all the nodes whose precedent nodes are all finished. It would be much
> better to call these nodes (which we would trigger) as "root nodes". I'll
> modify the FLIP to change the names to "root nodes".
>
> >> 2b) We also need to be careful for out-of-sync updates: if the root is
> about to finish, we could send the barrier to it from
> CheckpointCoordinator, but at the time it arrives, the subtask is finished
> already.
>
> Exactly. When the checkpoint triggers a task but found the task is not
> there, it may then further check if the task has been finished, if so, it
> should then re-check its descendants to see if there are new "root nodes"
> to trigger.
>
> >> 3) An implied change is that checkpoints are not aborted anymore at
> EndOfPartition, which is good, but might be explicitly added.
>
> Yes, currently barrier alignment would fail the current checkpoint on
> EndOfPartition, and we would modify the behavior.
>
> >> 4) The interaction between unaligned checkpoint and EndOfPartition is a
> bit ambiguous: What happens when an unaligned checkpoint is started and
> then one input channel contains the EndOfPartition event? From the written
> description, it sounds to me like, we move back to an aligned checkpoint
> for the whole receiving task. However, that is neither easily possible nor
> necessary. Imho it would be enough to also store the EndOfPartition in the
> channel state.
>
>
> Very thanks for the suggestions on this issue and in fact I did stuck on
> it for some time. Previously for me one implementation detail issue is that
> EndOfPartition seems not be able to overtake the previous buffers easily as
> CheckpointBarrier does, otherwise it might start destroying the input
> channels if all EndOfPartitions are received.
>
> Therefore, although we could also persistent the channels with
> EndOfPartition:
>
> 1. Start persisting the channels when CheckpointUnaligner received barrier
> (if not all precendant tasks are finished) or received triggering (if all
> precendant tasks are finished).
>
> 2. The persisting actually stops when onBuffer received EndOfPartition.
>
> After the last channel stopped persisting, CheckpointUnaligner still need
> to wait till all the previous buffers are processed before complete the
> allBarriersReceivedFuture. Therefore it would not be able to accelerate the
> checkpoint in this case.
>
> After some rethinking today currently I think we might inserts some
> additional virtual events into receivedBuffer when received EndOfPartition
> and allows these virtual events to overtake the previous buffers. I'll try
> to double check if it is feasible and let me know if there are also other
> solutions on this issue :).
>
> > 5) I'd expand the recovery section a bit. It would be the first time
> that we recover an incomplete DAG. Afaik the subtasks are deployed before
> the state is recovered, so at some point, the subtasks either need to be
> removed again or maybe we could even avoid them being created in the first
> place.
>
> I also agree that finally we should not "restarted" the finished tasks in
> some way. It seems not start it in the first place would be better. We
> should be able to bookkeep additional information in the checkpoint meta
> about which operators are fully finished, and the scheduler could restore
> 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 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 solution gradually in the next steps.
>
>
> Best,
>
> Yun
>
> ------------------------------------------------------------------
> From:Arvid Heise <ar...@ververica.com>
> Send Time:2020 Oct. 12 (Mon.) 15:38
> To:Yun Gao <yu...@aliyun.com>
> Cc:Flink Dev <de...@flink.apache.org>; User-Flink <us...@flink.apache.org>
> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
> Hi Yun,
>
> Thank you for starting the discussion. This will solve one of the
> long-standing issues [1] that confuse users. I'm also a big fan of option
> 3. It is also a bit closer to Chandy-Lamport again.
>
> A couple of comments:
>
> 1) You call the tasks that get the barriers injected leaf nodes, which
> would make the sinks the root nodes. That is very similar to how graphs in
> relational algebra are labeled. However, I got the feeling that in Flink,
> we rather iterate from sources to sink, making the sources root nodes and
> the sinks the leaf nodes. However, I have no clue how it's done in similar
> cases, so please take that hint cautiously.
> 2) I'd make the algorithm to find the subtasks iterative and react in
> CheckpointCoordinator. Let's assume that we inject the barrier at all root
> subtasks (initially all sources). So in the iterative algorithm, whenever
> root A finishes, it looks at all connected subtasks B if they have any
> upstream task left. If not B becomes a new root. That would require to only
> touch a part of the job graph, but would require some callback from
> JobManager to CheckpointCoordinator.
> 2b) We also need to be careful for out-of-sync updates: if the root is
> about to finish, we could send the barrier to it from
> CheckpointCoordinator, but at the time it arrives, the subtask is finished
> already.
> 3) An implied change is that checkpoints are not aborted anymore at EndOfPartition,
> which is good, but might be explicitly added.
> 4) The interaction between unaligned checkpoint and EndOfPartition is a
> bit ambiguous: What happens when an unaligned checkpoint is started and
> then one input channel contains the EndOfPartition event? From the
> written description, it sounds to me like, we move back to an aligned
> checkpoint for the whole receiving task. However, that is neither easily
> possible nor necessary. Imho it would be enough to also store the EndOfPartition
> in the channel state.
> 5) I'd expand the recovery section a bit. It would be the first time that
> we recover an incomplete DAG. Afaik the subtasks are deployed before the
> state is recovered, so at some point, the subtasks either need to be
> removed again or maybe we could even avoid them being created in the first
> place.
>
> [1] https://issues.apache.org/jira/browse/FLINK-2491
>
> On Fri, Oct 9, 2020 at 8:22 AM Yun Gao <yu...@aliyun.com> wrote:
> Hi, devs & users
>
> Very sorry for the spoiled formats, I resent the discussion as follows.
>
>
> 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 tasks finished, which causes some problems for bounded or mixed jobs:
>         1.
> Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5].
>         2.
> The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
>
>
> Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6].
>
> Best,
> Yun
>
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> [2]
> https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
> [3]
> https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
> [4]
> https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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 <yu...@aliyun.com.INVALID>
> *Send Date:*Fri Oct 9 14:16:52 2020
> *Recipients:*Flink Dev <de...@flink.apache.org>, User-Flink <
> user@flink.apache.org>
> *Subject:*[DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
> 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 tasks finished, which causes some problems for bounded or mixed jobs:
>
> Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5].
>
> The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
>
> Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6].
> Best,
> Yun
>
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> [2]
> https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
> [3]
> https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
> [4]
> https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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
>
>
> --
>
> Arvid Heise | Senior Java Developer
>
> <https://www.ververica.com/>
>
> Follow us @VervericaData
>
> --
>
> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
> Conference
>
> Stream Processing | Event Driven | Real Time
>
> --
>
> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>
> --
> Ververica GmbH
> Registered at Amtsgericht Charlottenburg: HRB 158244 B
> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
> (Toni) Cheng
>
>
>

-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

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

Posted by Arvid Heise <ar...@ververica.com>.
Hi Yun,

4) Yes, the interaction is not trivial and also I have not completely
thought it through. But in general, I'm currently at the point where I
think that we also need non-checkpoint related events in unaligned
checkpoints. So just keep that in mind, that we might converge anyhow at
this point.

In general, what is helping in this case is to remember that there no
unaligned checkpoint barrier ever going to overtake EndOfPartition. So, we
can completely ignore the problem on how to store and restore output
buffers of a completed task (also important for the next point).

5) I think we are on the same page and I completely agree that for the
MVP/first version, it's completely fine to start and immediately stop. A
tad better would be even to not even start the procession loop.

On Mon, Oct 12, 2020 at 6:18 PM Yun Gao <yu...@aliyun.com> wrote:

> Hi Arvid,
>
> Very thanks for the insightful comments! I added the responses for this
> issue under the quota:
>
> >> 1) You call the tasks that get the barriers injected leaf nodes, which
> would make the > sinks the root nodes. That is very similar to how graphs
> in relational algebra are labeled. However, I got the feeling that in
> Flink, we rather iterate from sources to sink, making the sources root
> nodes and the sinks the leaf nodes. However, I have no clue how it's done
> in similar cases, so please take that hint cautiously.
>
> >> 2) I'd make the algorithm to find the subtasks iterative and react in
> CheckpointCoordinator. Let's assume that we inject the barrier at all root
> subtasks (initially all sources). So in the iterative algorithm, whenever
> root A finishes, it looks at all connected subtasks B if they have any
> upstream task left. If not B becomes a new root. That would require to only
> touch a part of the job graph, but would require some callback from
> JobManager to CheckpointCoordinator.
>
>
> I think I should have used a bad name of "leaf nodes", in fact I think we
> should have the same thoughts that we start with the source nodes to find
> all the nodes whose precedent nodes are all finished. It would be much
> better to call these nodes (which we would trigger) as "root nodes". I'll
> modify the FLIP to change the names to "root nodes".
>
> >> 2b) We also need to be careful for out-of-sync updates: if the root is
> about to finish, we could send the barrier to it from
> CheckpointCoordinator, but at the time it arrives, the subtask is finished
> already.
>
> Exactly. When the checkpoint triggers a task but found the task is not
> there, it may then further check if the task has been finished, if so, it
> should then re-check its descendants to see if there are new "root nodes"
> to trigger.
>
> >> 3) An implied change is that checkpoints are not aborted anymore at
> EndOfPartition, which is good, but might be explicitly added.
>
> Yes, currently barrier alignment would fail the current checkpoint on
> EndOfPartition, and we would modify the behavior.
>
> >> 4) The interaction between unaligned checkpoint and EndOfPartition is a
> bit ambiguous: What happens when an unaligned checkpoint is started and
> then one input channel contains the EndOfPartition event? From the written
> description, it sounds to me like, we move back to an aligned checkpoint
> for the whole receiving task. However, that is neither easily possible nor
> necessary. Imho it would be enough to also store the EndOfPartition in the
> channel state.
>
>
> Very thanks for the suggestions on this issue and in fact I did stuck on
> it for some time. Previously for me one implementation detail issue is that
> EndOfPartition seems not be able to overtake the previous buffers easily as
> CheckpointBarrier does, otherwise it might start destroying the input
> channels if all EndOfPartitions are received.
>
> Therefore, although we could also persistent the channels with
> EndOfPartition:
>
> 1. Start persisting the channels when CheckpointUnaligner received barrier
> (if not all precendant tasks are finished) or received triggering (if all
> precendant tasks are finished).
>
> 2. The persisting actually stops when onBuffer received EndOfPartition.
>
> After the last channel stopped persisting, CheckpointUnaligner still need
> to wait till all the previous buffers are processed before complete the
> allBarriersReceivedFuture. Therefore it would not be able to accelerate the
> checkpoint in this case.
>
> After some rethinking today currently I think we might inserts some
> additional virtual events into receivedBuffer when received EndOfPartition
> and allows these virtual events to overtake the previous buffers. I'll try
> to double check if it is feasible and let me know if there are also other
> solutions on this issue :).
>
> > 5) I'd expand the recovery section a bit. It would be the first time
> that we recover an incomplete DAG. Afaik the subtasks are deployed before
> the state is recovered, so at some point, the subtasks either need to be
> removed again or maybe we could even avoid them being created in the first
> place.
>
> I also agree that finally we should not "restarted" the finished tasks in
> some way. It seems not start it in the first place would be better. We
> should be able to bookkeep additional information in the checkpoint meta
> about which operators are fully finished, and the scheduler could restore
> 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 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 solution gradually in the next steps.
>
>
> Best,
>
> Yun
>
> ------------------------------------------------------------------
> From:Arvid Heise <ar...@ververica.com>
> Send Time:2020 Oct. 12 (Mon.) 15:38
> To:Yun Gao <yu...@aliyun.com>
> Cc:Flink Dev <de...@flink.apache.org>; User-Flink <us...@flink.apache.org>
> Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
> Hi Yun,
>
> Thank you for starting the discussion. This will solve one of the
> long-standing issues [1] that confuse users. I'm also a big fan of option
> 3. It is also a bit closer to Chandy-Lamport again.
>
> A couple of comments:
>
> 1) You call the tasks that get the barriers injected leaf nodes, which
> would make the sinks the root nodes. That is very similar to how graphs in
> relational algebra are labeled. However, I got the feeling that in Flink,
> we rather iterate from sources to sink, making the sources root nodes and
> the sinks the leaf nodes. However, I have no clue how it's done in similar
> cases, so please take that hint cautiously.
> 2) I'd make the algorithm to find the subtasks iterative and react in
> CheckpointCoordinator. Let's assume that we inject the barrier at all root
> subtasks (initially all sources). So in the iterative algorithm, whenever
> root A finishes, it looks at all connected subtasks B if they have any
> upstream task left. If not B becomes a new root. That would require to only
> touch a part of the job graph, but would require some callback from
> JobManager to CheckpointCoordinator.
> 2b) We also need to be careful for out-of-sync updates: if the root is
> about to finish, we could send the barrier to it from
> CheckpointCoordinator, but at the time it arrives, the subtask is finished
> already.
> 3) An implied change is that checkpoints are not aborted anymore at EndOfPartition,
> which is good, but might be explicitly added.
> 4) The interaction between unaligned checkpoint and EndOfPartition is a
> bit ambiguous: What happens when an unaligned checkpoint is started and
> then one input channel contains the EndOfPartition event? From the
> written description, it sounds to me like, we move back to an aligned
> checkpoint for the whole receiving task. However, that is neither easily
> possible nor necessary. Imho it would be enough to also store the EndOfPartition
> in the channel state.
> 5) I'd expand the recovery section a bit. It would be the first time that
> we recover an incomplete DAG. Afaik the subtasks are deployed before the
> state is recovered, so at some point, the subtasks either need to be
> removed again or maybe we could even avoid them being created in the first
> place.
>
> [1] https://issues.apache.org/jira/browse/FLINK-2491
>
> On Fri, Oct 9, 2020 at 8:22 AM Yun Gao <yu...@aliyun.com> wrote:
> Hi, devs & users
>
> Very sorry for the spoiled formats, I resent the discussion as follows.
>
>
> 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 tasks finished, which causes some problems for bounded or mixed jobs:
>         1.
> Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5].
>         2.
> The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
>
>
> Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6].
>
> Best,
> Yun
>
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> [2]
> https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
> [3]
> https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
> [4]
> https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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 <yu...@aliyun.com.INVALID>
> *Send Date:*Fri Oct 9 14:16:52 2020
> *Recipients:*Flink Dev <de...@flink.apache.org>, User-Flink <
> user@flink.apache.org>
> *Subject:*[DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
> 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 tasks finished, which causes some problems for bounded or mixed jobs:
>
> Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5].
>
> The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
>
> Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6].
> Best,
> Yun
>
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> [2]
> https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
> [3]
> https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
> [4]
> https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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
>
>
> --
>
> Arvid Heise | Senior Java Developer
>
> <https://www.ververica.com/>
>
> Follow us @VervericaData
>
> --
>
> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
> Conference
>
> Stream Processing | Event Driven | Real Time
>
> --
>
> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>
> --
> Ververica GmbH
> Registered at Amtsgericht Charlottenburg: HRB 158244 B
> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
> (Toni) Cheng
>
>
>

-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi Arvid,
Very thanks for the insightful comments! I added the responses for this issue under the quota: 
>> 1) You call the tasks that get the barriers injected leaf nodes, which would make the > sinks the root nodes. That is very similar to how graphs in relational algebra are labeled. However, I got the feeling that in Flink, we rather iterate from sources to sink, making the sources root nodes and the sinks the leaf nodes. However, I have no clue how it's done in similar cases, so please take that hint cautiously.
>> 2) I'd make the algorithm to find the subtasks iterative and react in CheckpointCoordinator. Let's assume that we inject the barrier at all root subtasks (initially all sources). So in the iterative algorithm, whenever root A finishes, it looks at all connected subtasks B if they have any upstream task left. If not B becomes a new root. That would require to only touch a part of the job graph, but would require some callback from JobManager to CheckpointCoordinator.

I think I should have used a bad name of "leaf nodes", in fact I think we should have the same thoughts that we start with the source nodes to find all the nodes whose precedent nodes are all finished. It would be much better to call these nodes (which we would trigger) as "root nodes". I'll modify the FLIP to change the names to "root nodes".
>> 2b) We also need to be careful for out-of-sync updates: if the root is about to finish, we could send the barrier to it from CheckpointCoordinator, but at the time it arrives, the subtask is finished already.
Exactly. When the checkpoint triggers a task but found the task is not there, it may then further check if the task has been finished, if so, it should then re-check its descendants to see if there are new "root nodes" to trigger.
>> 3) An implied change is that checkpoints are not aborted anymore at EndOfPartition, which is good, but might be explicitly added.
Yes, currently barrier alignment would fail the current checkpoint on EndOfPartition, and we would modify the behavior.
>> 4) The interaction between unaligned checkpoint and EndOfPartition is a bit ambiguous: What happens when an unaligned checkpoint is started and then one input channel contains the EndOfPartition event? From the written description, it sounds to me like, we move back to an aligned checkpoint for the whole receiving task. However, that is neither easily possible nor necessary. Imho it would be enough to also store the EndOfPartition in the channel state.

Very thanks for the suggestions on this issue and in fact I did stuck on it for some time. Previously for me one implementation detail issue is that EndOfPartition seems not be able to overtake the previous buffers easily as CheckpointBarrier does, otherwise it might start destroying the input channels if all EndOfPartitions are received.
Therefore, although we could also persistent the channels with EndOfPartition:
1. Start persisting the channels when CheckpointUnaligner received barrier (if not all precendant tasks are finished) or received triggering (if all precendant tasks are finished).
2. The persisting actually stops when onBuffer received EndOfPartition.
After the last channel stopped persisting, CheckpointUnaligner still need to wait till all the previous buffers are processed before complete the allBarriersReceivedFuture. Therefore it would not be able to accelerate the checkpoint in this case.
After some rethinking today currently I think we might inserts some additional virtual events into receivedBuffer when received EndOfPartition and allows these virtual events to overtake the previous buffers. I'll try to double check if it is feasible and let me know if there are also other solutions on this issue :). 
> 5) I'd expand the recovery section a bit. It would be the first time that we recover an incomplete DAG. Afaik the subtasks are deployed before the state is recovered, so at some point, the subtasks either need to be removed again or maybe we could even avoid them being created in the first place.
I also agree that finally we should not "restarted" the finished tasks in some way. It seems not start it in the first place would be better. We should be able to bookkeep additional information in the checkpoint meta about which operators are fully finished, and the scheduler could restore 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 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 solution gradually in the next steps. 

Best,
Yun


------------------------------------------------------------------
From:Arvid Heise <ar...@ververica.com>
Send Time:2020 Oct. 12 (Mon.) 15:38
To:Yun Gao <yu...@aliyun.com>
Cc:Flink Dev <de...@flink.apache.org>; User-Flink <us...@flink.apache.org>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi Yun,

Thank you for starting the discussion. This will solve one of the long-standing issues [1] that confuse users. I'm also a big fan of option 3. It is also a bit closer to Chandy-Lamport again.

A couple of comments:

1) You call the tasks that get the barriers injected leaf nodes, which would make the sinks the root nodes. That is very similar to how graphs in relational algebra are labeled. However, I got the feeling that in Flink, we rather iterate from sources to sink, making the sources root nodes and the sinks the leaf nodes. However, I have no clue how it's done in similar cases, so please take that hint cautiously.
2) I'd make the algorithm to find the subtasks iterative and react in CheckpointCoordinator. Let's assume that we inject the barrier at all root subtasks (initially all sources). So in the iterative algorithm, whenever root A finishes, it looks at all connected subtasks B if they have any upstream task left. If not B becomes a new root. That would require to only touch a part of the job graph, but would require some callback from JobManager to CheckpointCoordinator. 
2b) We also need to be careful for out-of-sync updates: if the root is about to finish, we could send the barrier to it from CheckpointCoordinator, but at the time it arrives, the subtask is finished already.
3) An implied change is that checkpoints are not aborted anymore at EndOfPartition, which is good, but might be explicitly added.
4) The interaction between unaligned checkpoint and EndOfPartition is a bit ambiguous: What happens when an unaligned checkpoint is started and then one input channel contains the EndOfPartition event? From the written description, it sounds to me like, we move back to an aligned checkpoint for the whole receiving task. However, that is neither easily possible nor necessary. Imho it would be enough to also store the EndOfPartition in the channel state.
5) I'd expand the recovery section a bit. It would be the first time that we recover an incomplete DAG. Afaik the subtasks are deployed before the state is recovered, so at some point, the subtasks either need to be removed again or maybe we could even avoid them being created in the first place.

[1] https://issues.apache.org/jira/browse/FLINK-2491
On Fri, Oct 9, 2020 at 8:22 AM Yun Gao <yu...@aliyun.com> wrote:

Hi, devs & users

Very sorry for the spoiled formats, I resent the discussion as follows.

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 tasks finished, which causes some problems for bounded or mixed jobs:
        1. Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5]. 
        2. The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.

Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6]. 

Best,
Yun

[1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
[2] https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
[3] https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
[4] https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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 <yu...@aliyun.com.INVALID>
Send Date:Fri Oct 9 14:16:52 2020
Recipients:Flink Dev <de...@flink.apache.org>, User-Flink <us...@flink.apache.org>
Subject:[DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
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 tasks finished, which causes some problems for bounded or mixed jobs:
Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5]. 
The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6]. 
Best,
Yun

[1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
[2] https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
[3] https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
[4] https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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

-- 
Arvid Heise | Senior Java Developer

Follow us @VervericaData
--
Join Flink Forward - The Apache Flink Conference
Stream Processing | Event Driven | Real Time
--
Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
--Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji (Toni) Cheng


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

Posted by Yun Gao <yu...@aliyun.com>.
Hi Arvid,
Very thanks for the insightful comments! I added the responses for this issue under the quota: 
>> 1) You call the tasks that get the barriers injected leaf nodes, which would make the > sinks the root nodes. That is very similar to how graphs in relational algebra are labeled. However, I got the feeling that in Flink, we rather iterate from sources to sink, making the sources root nodes and the sinks the leaf nodes. However, I have no clue how it's done in similar cases, so please take that hint cautiously.
>> 2) I'd make the algorithm to find the subtasks iterative and react in CheckpointCoordinator. Let's assume that we inject the barrier at all root subtasks (initially all sources). So in the iterative algorithm, whenever root A finishes, it looks at all connected subtasks B if they have any upstream task left. If not B becomes a new root. That would require to only touch a part of the job graph, but would require some callback from JobManager to CheckpointCoordinator.

I think I should have used a bad name of "leaf nodes", in fact I think we should have the same thoughts that we start with the source nodes to find all the nodes whose precedent nodes are all finished. It would be much better to call these nodes (which we would trigger) as "root nodes". I'll modify the FLIP to change the names to "root nodes".
>> 2b) We also need to be careful for out-of-sync updates: if the root is about to finish, we could send the barrier to it from CheckpointCoordinator, but at the time it arrives, the subtask is finished already.
Exactly. When the checkpoint triggers a task but found the task is not there, it may then further check if the task has been finished, if so, it should then re-check its descendants to see if there are new "root nodes" to trigger.
>> 3) An implied change is that checkpoints are not aborted anymore at EndOfPartition, which is good, but might be explicitly added.
Yes, currently barrier alignment would fail the current checkpoint on EndOfPartition, and we would modify the behavior.
>> 4) The interaction between unaligned checkpoint and EndOfPartition is a bit ambiguous: What happens when an unaligned checkpoint is started and then one input channel contains the EndOfPartition event? From the written description, it sounds to me like, we move back to an aligned checkpoint for the whole receiving task. However, that is neither easily possible nor necessary. Imho it would be enough to also store the EndOfPartition in the channel state.

Very thanks for the suggestions on this issue and in fact I did stuck on it for some time. Previously for me one implementation detail issue is that EndOfPartition seems not be able to overtake the previous buffers easily as CheckpointBarrier does, otherwise it might start destroying the input channels if all EndOfPartitions are received.
Therefore, although we could also persistent the channels with EndOfPartition:
1. Start persisting the channels when CheckpointUnaligner received barrier (if not all precendant tasks are finished) or received triggering (if all precendant tasks are finished).
2. The persisting actually stops when onBuffer received EndOfPartition.
After the last channel stopped persisting, CheckpointUnaligner still need to wait till all the previous buffers are processed before complete the allBarriersReceivedFuture. Therefore it would not be able to accelerate the checkpoint in this case.
After some rethinking today currently I think we might inserts some additional virtual events into receivedBuffer when received EndOfPartition and allows these virtual events to overtake the previous buffers. I'll try to double check if it is feasible and let me know if there are also other solutions on this issue :). 
> 5) I'd expand the recovery section a bit. It would be the first time that we recover an incomplete DAG. Afaik the subtasks are deployed before the state is recovered, so at some point, the subtasks either need to be removed again or maybe we could even avoid them being created in the first place.
I also agree that finally we should not "restarted" the finished tasks in some way. It seems not start it in the first place would be better. We should be able to bookkeep additional information in the checkpoint meta about which operators are fully finished, and the scheduler could restore 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 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 solution gradually in the next steps. 

Best,
Yun


------------------------------------------------------------------
From:Arvid Heise <ar...@ververica.com>
Send Time:2020 Oct. 12 (Mon.) 15:38
To:Yun Gao <yu...@aliyun.com>
Cc:Flink Dev <de...@flink.apache.org>; User-Flink <us...@flink.apache.org>
Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

Hi Yun,

Thank you for starting the discussion. This will solve one of the long-standing issues [1] that confuse users. I'm also a big fan of option 3. It is also a bit closer to Chandy-Lamport again.

A couple of comments:

1) You call the tasks that get the barriers injected leaf nodes, which would make the sinks the root nodes. That is very similar to how graphs in relational algebra are labeled. However, I got the feeling that in Flink, we rather iterate from sources to sink, making the sources root nodes and the sinks the leaf nodes. However, I have no clue how it's done in similar cases, so please take that hint cautiously.
2) I'd make the algorithm to find the subtasks iterative and react in CheckpointCoordinator. Let's assume that we inject the barrier at all root subtasks (initially all sources). So in the iterative algorithm, whenever root A finishes, it looks at all connected subtasks B if they have any upstream task left. If not B becomes a new root. That would require to only touch a part of the job graph, but would require some callback from JobManager to CheckpointCoordinator. 
2b) We also need to be careful for out-of-sync updates: if the root is about to finish, we could send the barrier to it from CheckpointCoordinator, but at the time it arrives, the subtask is finished already.
3) An implied change is that checkpoints are not aborted anymore at EndOfPartition, which is good, but might be explicitly added.
4) The interaction between unaligned checkpoint and EndOfPartition is a bit ambiguous: What happens when an unaligned checkpoint is started and then one input channel contains the EndOfPartition event? From the written description, it sounds to me like, we move back to an aligned checkpoint for the whole receiving task. However, that is neither easily possible nor necessary. Imho it would be enough to also store the EndOfPartition in the channel state.
5) I'd expand the recovery section a bit. It would be the first time that we recover an incomplete DAG. Afaik the subtasks are deployed before the state is recovered, so at some point, the subtasks either need to be removed again or maybe we could even avoid them being created in the first place.

[1] https://issues.apache.org/jira/browse/FLINK-2491
On Fri, Oct 9, 2020 at 8:22 AM Yun Gao <yu...@aliyun.com> wrote:

Hi, devs & users

Very sorry for the spoiled formats, I resent the discussion as follows.

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 tasks finished, which causes some problems for bounded or mixed jobs:
        1. Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5]. 
        2. The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.

Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6]. 

Best,
Yun

[1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
[2] https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
[3] https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
[4] https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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 <yu...@aliyun.com.INVALID>
Send Date:Fri Oct 9 14:16:52 2020
Recipients:Flink Dev <de...@flink.apache.org>, User-Flink <us...@flink.apache.org>
Subject:[DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
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 tasks finished, which causes some problems for bounded or mixed jobs:
Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5]. 
The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6]. 
Best,
Yun

[1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
[2] https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
[3] https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
[4] https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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

-- 
Arvid Heise | Senior Java Developer

Follow us @VervericaData
--
Join Flink Forward - The Apache Flink Conference
Stream Processing | Event Driven | Real Time
--
Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
--Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji (Toni) Cheng


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

Posted by Arvid Heise <ar...@ververica.com>.
Hi Yun,

Thank you for starting the discussion. This will solve one of the
long-standing issues [1] that confuse users. I'm also a big fan of option
3. It is also a bit closer to Chandy-Lamport again.

A couple of comments:

1) You call the tasks that get the barriers injected leaf nodes, which
would make the sinks the root nodes. That is very similar to how graphs in
relational algebra are labeled. However, I got the feeling that in Flink,
we rather iterate from sources to sink, making the sources root nodes and
the sinks the leaf nodes. However, I have no clue how it's done in similar
cases, so please take that hint cautiously.
2) I'd make the algorithm to find the subtasks iterative and react in
CheckpointCoordinator. Let's assume that we inject the barrier at all root
subtasks (initially all sources). So in the iterative algorithm, whenever
root A finishes, it looks at all connected subtasks B if they have any
upstream task left. If not B becomes a new root. That would require to only
touch a part of the job graph, but would require some callback from
JobManager to CheckpointCoordinator.
2b) We also need to be careful for out-of-sync updates: if the root is
about to finish, we could send the barrier to it from
CheckpointCoordinator, but at the time it arrives, the subtask is finished
already.
3) An implied change is that checkpoints are not aborted anymore at
EndOfPartition,
which is good, but might be explicitly added.
4) The interaction between unaligned checkpoint and EndOfPartition is a bit
ambiguous: What happens when an unaligned checkpoint is started and then
one input channel contains the EndOfPartition event? From the written
description, it sounds to me like, we move back to an aligned checkpoint
for the whole receiving task. However, that is neither easily possible nor
necessary. Imho it would be enough to also store the EndOfPartition in the
channel state.
5) I'd expand the recovery section a bit. It would be the first time that
we recover an incomplete DAG. Afaik the subtasks are deployed before the
state is recovered, so at some point, the subtasks either need to be
removed again or maybe we could even avoid them being created in the first
place.

[1] https://issues.apache.org/jira/browse/FLINK-2491

On Fri, Oct 9, 2020 at 8:22 AM Yun Gao <yu...@aliyun.com> wrote:

> Hi, devs & users
>
> Very sorry for the spoiled formats, I resent the discussion as follows.
>
>
> 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 s
> ​
> upport checkpoints after some tasks finished, which causes some problems for bounded or mixed jobs:
>         1.
> Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5].
>         2.
> The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
>
>
> Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6].
>
> Best,
> Yun
>
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> [2]
> https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
> [3]
> https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
> [4]
> https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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 <yu...@aliyun.com.INVALID>
> *Send Date:*Fri Oct 9 14:16:52 2020
> *Recipients:*Flink Dev <de...@flink.apache.org>, User-Flink <
> user@flink.apache.org>
> *Subject:*[DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
>> 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 tasks finished, which causes some problems for bounded or mixed jobs:
>>
>> Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5].
>>
>> The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
>>
>> Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6].
>> Best,
>> Yun
>>
>> [1]
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
>> [2]
>> https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
>> [3]
>> https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
>> [4]
>> https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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
>
>

-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

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

Posted by Arvid Heise <ar...@ververica.com>.
Hi Yun,

Thank you for starting the discussion. This will solve one of the
long-standing issues [1] that confuse users. I'm also a big fan of option
3. It is also a bit closer to Chandy-Lamport again.

A couple of comments:

1) You call the tasks that get the barriers injected leaf nodes, which
would make the sinks the root nodes. That is very similar to how graphs in
relational algebra are labeled. However, I got the feeling that in Flink,
we rather iterate from sources to sink, making the sources root nodes and
the sinks the leaf nodes. However, I have no clue how it's done in similar
cases, so please take that hint cautiously.
2) I'd make the algorithm to find the subtasks iterative and react in
CheckpointCoordinator. Let's assume that we inject the barrier at all root
subtasks (initially all sources). So in the iterative algorithm, whenever
root A finishes, it looks at all connected subtasks B if they have any
upstream task left. If not B becomes a new root. That would require to only
touch a part of the job graph, but would require some callback from
JobManager to CheckpointCoordinator.
2b) We also need to be careful for out-of-sync updates: if the root is
about to finish, we could send the barrier to it from
CheckpointCoordinator, but at the time it arrives, the subtask is finished
already.
3) An implied change is that checkpoints are not aborted anymore at
EndOfPartition,
which is good, but might be explicitly added.
4) The interaction between unaligned checkpoint and EndOfPartition is a bit
ambiguous: What happens when an unaligned checkpoint is started and then
one input channel contains the EndOfPartition event? From the written
description, it sounds to me like, we move back to an aligned checkpoint
for the whole receiving task. However, that is neither easily possible nor
necessary. Imho it would be enough to also store the EndOfPartition in the
channel state.
5) I'd expand the recovery section a bit. It would be the first time that
we recover an incomplete DAG. Afaik the subtasks are deployed before the
state is recovered, so at some point, the subtasks either need to be
removed again or maybe we could even avoid them being created in the first
place.

[1] https://issues.apache.org/jira/browse/FLINK-2491

On Fri, Oct 9, 2020 at 8:22 AM Yun Gao <yu...@aliyun.com> wrote:

> Hi, devs & users
>
> Very sorry for the spoiled formats, I resent the discussion as follows.
>
>
> 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 s
> ​
> upport checkpoints after some tasks finished, which causes some problems for bounded or mixed jobs:
>         1.
> Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5].
>         2.
> The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
>
>
> Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6].
>
> Best,
> Yun
>
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> [2]
> https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
> [3]
> https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
> [4]
> https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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 <yu...@aliyun.com.INVALID>
> *Send Date:*Fri Oct 9 14:16:52 2020
> *Recipients:*Flink Dev <de...@flink.apache.org>, User-Flink <
> user@flink.apache.org>
> *Subject:*[DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
>
>> 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 tasks finished, which causes some problems for bounded or mixed jobs:
>>
>> Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5].
>>
>> The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
>>
>> Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6].
>> Best,
>> Yun
>>
>> [1]
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
>> [2]
>> https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
>> [3]
>> https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
>> [4]
>> https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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
>
>

-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

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

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi, devs & users

Very sorry for the spoiled formats, I resent the discussion as follows.

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 s​upport checkpoints after some tasks finished, which causes some problems for bounded or mixed jobs:
        1. Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5]. 
        2. The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.

Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6]. 

Best,
Yun

[1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
[2] https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
[3] https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
[4] https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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 <yu...@aliyun.com.INVALID>
Send Date:Fri Oct 9 14:16:52 2020
Recipients:Flink Dev <de...@flink.apache.org>, User-Flink <us...@flink.apache.org>
Subject:[DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
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 tasks finished, which causes some problems for bounded or mixed jobs:
Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5]. 
The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6]. 
Best,
Yun

[1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
[2] https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
[3] https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
[4] https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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

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

Posted by Aljoscha Krettek <al...@apache.org>.
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 least for Async I/O and 
potentially for sinks.

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 interface would have a method like `isFinalized()` that the 
framework can call after each checkpoint (and potentially at other 
points)

This way we would decouple that logic from things that don't actually 
need it. What do you think?

Best,
Aljoscha

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

Posted by Yun Gao <yu...@aliyun.com>.
Hi, devs & users

Very sorry for the spoiled formats, I resent the discussion as follows.

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 s​upport checkpoints after some tasks finished, which causes some problems for bounded or mixed jobs:
        1. Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5]. 
        2. The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.

Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6]. 

Best,
Yun

[1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
[2] https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
[3] https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
[4] https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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 <yu...@aliyun.com.INVALID>
Send Date:Fri Oct 9 14:16:52 2020
Recipients:Flink Dev <de...@flink.apache.org>, User-Flink <us...@flink.apache.org>
Subject:[DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished
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 tasks finished, which causes some problems for bounded or mixed jobs:
Flink exactly-once sinks rely on checkpoints to ensure data won’t be replayed before committed to external systems in streaming mode. If sources are bounded and checkpoints are disabled after some tasks are finished, the data sent after the last checkpoint would always not be able to be committed. This issue has already been reported some times in the user ML[2][3][4] and is future brought up when working on FLIP-143: Unified Sink API [5]. 
The jobs with both bounded and unbounded sources might have to replay a large amount of records after failover due to no periodic checkpoints are taken after the bounded sources finished.
Therefore, we propose to also support checkpoints after some tasks finished. Your Could find more details in FLIP-147[6]. 
Best,
Yun

[1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
[2] https://lists.apache.org/thread.html/rea1ac2d82f646fcea1395b5738be495f144c5b0312a290a1d4a339c1%40%3Cuser.flink.apache.org%3E
[3] https://lists.apache.org/thread.html/rad4adeec838093b8b56ae9e2ea6a937a4b1882b53045a12acb7e61ea%40%3Cuser.flink.apache.org%3E
[4] https://lists.apache.org/thread.html/4cf28a9fa3732dfdd9e673da6233c5288ca80b20d58cee130bf1c141%40%3Cuser.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