I agree and already created a Jira issue for removing the old “stop” feature as preparation: https://issues.apache.org/jira/browse/FLINK-11889 <https://issues.apache.org/jira/browse/FLINK-11889>
Aljoscha > On 7. Mar 2019, at 11:08, Kostas Kloudas <kklou...@gmail.com> wrote: > > Hi, > > Thanks for the comments. > I agree with the Ufuk's and Elias' proposal. > > - "cancel" remains the good old "cancel" > - "terminate" becomes "stop --drain-with-savepoint" > - "suspend" becomes "stop --with-savepoint" > - "cancel-with-savepoint" is subsumed by "stop --with-savepoint" > > As you see from the previous, I would also add "terminate" and "suspend" > to result in keeping a savepoint by default. > > As for Ufuk's remarks: > > 1) You are correct that to have a proper way to not allow elements to be > fed in the pipeline > after the checkpoint barrier, we need support from the sources. This is > more the responsibility > of FLIP-27 > https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface > > 2) I would lean more towards replacing the old "stop" command with the new > one. But, as you said, > I have no view of how many users (if any) rely on the old "stop" command > for their usecases. > > Cheers, > Kostas > > > > On Wed, Mar 6, 2019 at 9:52 PM Ufuk Celebi <u...@apache.org> wrote: > >> I really like this effort. I think the original plan for >> "cancel-with-savepoint" was always to just be a workaround until we >> arrived at a better solution as proposed here. >> >> Regarding the FLIP, I agree with Elias comments. I think the number of >> termination modes the FLIP introduces can be overwhelming and I would >> personally rather follow Elias' proposal. In context of the proposal, >> this would result in the following: >> - "terminate" becomes "stop --drain" >> - "suspend" becomes "stop --with-savepoint" >> - "cancel-with-savepoint" is superseded by "stop --with-savepoint" >> >> I have two remaining questions: >> >> 1) @Kostas: Elias suggests for stop that "a job should process no >> messages after the checkpoints barrier". This is something that needs >> support from the sources. Is this in the scope of your proposal (I >> think not)? If not, is there a future plan for this? >> >> 2) Would we need to introduce a new command/name for "stop" as we >> already have a "stop" command? Assuming that there are no users that >> actually use the existing "stop" command as no major sources are >> stoppable (I think), I would personally suggest to upgrade the >> existing "stop" command to the proposed one. If on the other hand, if >> we know of users that rely on the current "stop" command, we'd need to >> find another name for it. >> >> Best, >> >> Ufuk >> >> On Wed, Mar 6, 2019 at 12:27 AM Elias Levy <fearsome.lucid...@gmail.com> >> wrote: >>> >>> Apologies for the late reply. >>> >>> I think this is badly needed, but I fear we are adding complexity by >>> introducing yet two more stop commands. We'll have: cancel, stop, >>> terminate. and suspend. We basically want to do two things: terminate a >>> job with prejudice or stop a job safely. >>> >>> For the former "cancel" is the appropriate term, and should have no need >>> for a cancel with checkpoint option. If the job was configured to use >>> externalized checkpoints and it ran long enough, a checkpoint will be >>> available for it. >>> >>> For the later "stop" is the appropriate term, and it means that a job >>> should process no messages after the checkpoints barrier and that it >> should >>> ensure that exactly-once sinks complete their two-phase commits >>> successfully. If a savepoint was requested, one should be created. >>> >>> So in my mind there are two commands, cancel and stop, with appropriate >>> semantics. Emitting MAX_WATERMARK before the checkpoint barrier during >>> stop is merely an optional behavior, like creation of a savepoint. But >> if >>> a specific command for it is desired, then "drain" seems appropriate. >>> >>> On Tue, Feb 12, 2019 at 9:50 AM Stephan Ewen <se...@apache.org> wrote: >>> >>>> Hi Elias! >>>> >>>> I remember you brought this missing feature up in the past. Do you >> think >>>> the proposed enhancement would work for your use case? >>>> >>>> Best, >>>> Stephan >>>> >>>> ---------- Forwarded message --------- >>>> From: Kostas Kloudas <k.klou...@ververica.com> >>>> Date: Tue, Feb 12, 2019 at 5:28 PM >>>> Subject: [DISCUSS] FLIP-33: Terminate/Suspend Job with Savepoint >>>> To: <dev@flink.apache.org> >>>> >>>> >>>> Hi everyone, >>>> >>>> A commonly used functionality offered by Flink is the >>>> "cancel-with-savepoint" operation. When applied to the current >> exactly-once >>>> sinks, the current implementation of the feature can be problematic, >> as it >>>> does not guarantee that side-effects will be committed by Flink to the >> 3rd >>>> party storage system. >>>> >>>> This discussion targets fixing this issue and proposes the addition >> of two >>>> termination modes, namely: >>>> 1) SUSPEND, for temporarily stopping the job, e.g. for Flink >> version >>>> upgrading in your cluster >>>> 2) TERMINATE, for terminal shut down which ends the stream and >> sends >>>> MAX_WATERMARK time, and flushes any state associated with (event time) >>>> timers >>>> >>>> A google doc with the FLIP proposal can be found here: >>>> >>>> >> https://docs.google.com/document/d/1EZf6pJMvqh_HeBCaUOnhLUr9JmkhfPgn6Mre_z6tgp8/edit?usp=sharing >>>> >>>> And the page for the FLIP is here: >>>> >> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=103090212 >>>> >>>> The implementation sketch is far from complete, but it is worth >> having a >>>> discussion on the semantics as soon as possible. The implementation >> section >>>> is going to be updated soon. >>>> >>>> Looking forward to the discussion, >>>> Kostas >>>> >>>> -- >>>> >>>> Kostas Kloudas | Software Engineer >>>> >>>> >>>> <https://www.ververica.com/> >>>> >>>> Follow us @VervericaData >>>> >>>> -- >>>> >>>> Join Flink Forward <https://flink-forward.org/> - The Apache Flink >>>> Conference >>>> >>>> Stream Processing | Event Driven | Real Time >>>> >>>> -- >>>> >>>> Data Artisans GmbH | Invalidenstrasse 115, 10115 Berlin, Germany >>>> >>>> -- >>>> Data Artisans GmbH >>>> Registered at Amtsgericht Charlottenburg: HRB 158244 B >>>> Managing Directors: Dr. Kostas Tzoumas, Dr. Stephan Ewen >>>> >>