re are new issues I'll post here and we could have some
> > more discussion.
> >
> > Best,
> > Yun Gao
> >
> >
> > --Original Mail --
> > *Sender:*Dawid Wysakowicz
> >
> > *Send Date:*Fri Dec 2 21:21:25 2022
; Yun Gao
>
>
> --Original Mail --
> *Sender:*Dawid Wysakowicz
>
> *Send Date:*Fri Dec 2 21:21:25 2022
> *Recipients:*Dev
> *Subject:*Re: [DISCUSS] FLIP-269: Properly Handling the Processing Timers
> on Job Termination
>
>> Ad. 1
&g
on.
Best,
Yun Gao
--Original Mail --
*Sender:*Dawid Wysakowicz
*Send Date:*Fri Dec 2 21:21:25 2022
*Recipients:*Dev
*Subject:*Re: [DISCUSS] FLIP-269: Properly Handling the Processing
Timers on Job Termination
Ad. 1
ginal Mail --
Sender:Dawid Wysakowicz
Send Date:Fri Dec 2 21:21:25 2022
Recipients:Dev
Subject:Re: [DISCUSS] FLIP-269: Properly Handling the Processing Timers on Job
Termination
Ad. 1
I'd start with ProcessingTimerService as that's the only public interface. It
is exposed in the Sink V2 interface. In thi
--
From:Piotr Nowojski
Send Time:2022 Nov. 30 (Wed.) 17:10
To:dev
Subject:Re: [DISCUSS] FLIP-269: Properly Handling the Processing Timers on Job
Termination
Hi,
I have a couple of remarks.
First a general one. For me the important part in the design of this API is
how to expose this to Flink
Subject:Re: [DISCUSS] FLIP-269: Properly Handling the Processing Timers on Job
Termination
Hi,
I have a couple of remarks.
First a general one. For me the important part in the design of this API is
how to expose this to Flink users in public interfaces. Namely
ProcessFunction and StreamOperator
t; similar to the proposed one that drain all the times and process it.
> > Best,
> > Yun Gao
> > [1]
> https://github.com/apache/flink/blob/a7fdab8b23cddf568fa32ee7eb804d7c3eb23a35/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/Win
488
>
--
From:Dawid Wysakowicz
Send Time:2022 Nov. 28 (Mon.) 23:33
To:dev
Subject:Re: [DISCUSS] FLIP-269: Properly Handling the Processing Timers on Job
Termination
Do we really need to have separate methods for triggering/waiting/cancelling.
To me it sounds
-
From:Dawid Wysakowicz
Send Time:2022 Nov. 28 (Mon.) 23:33
To:dev
Subject:Re: [DISCUSS] FLIP-269: Properly Handling the Processing Timers on Job
Termination
Do we really need to have separate methods for triggering/waiting/cancelling.
To me it sounds rathe
t,
Yun Gao
--
From:Divye Kapoor
Send Time:2022 Nov. 24 (Thu.) 08:50
To:dev
Cc:Xenon Development Team
Subject:Re: Re: [DISCUSS] FLIP-269: Properly Handling the Processing Timers on
Job Termination
Sounds good. Looks like we're on the same page.
Thanks!
Divye
On Wed, Nov 23, 2022 at 2:41 AM Piot
o support the callback options would be more complex.
Best,
Yun Gao
--
From:Divye Kapoor
Send Time:2022 Nov. 24 (Thu.) 08:50
To:dev
Cc:Xenon Development Team
Subject:Re: Re: [DISCUSS] FLIP-269: Properly Handling the Processing Timer
Sounds good. Looks like we're on the same page.
Thanks!
Divye
On Wed, Nov 23, 2022 at 2:41 AM Piotr Nowojski wrote:
> Hi Divye
>
> I think we are mostly on the same page. Just to clarify/rephrase:
>
> > One thing to think about - on EOF “trigger immediately” will mean that
> the
> >
Hi Divye
I think we are mostly on the same page. Just to clarify/rephrase:
> One thing to think about - on EOF “trigger immediately” will mean that the
> asynchronous wait timeout timers will also fire - which is undesirable
I didn't mean to fire all timers immediately in all of the built-in
Thanks Yun/Piotrek,
Some brief comments inline below.
On Tue, Nov 22, 2022 at 1:37 AM Piotr Nowojski wrote:
> Hi,
>
> All in all I would agree with Dawid's proposal.
+1
We can add the flexibility
> of how to deal with the timers in the low level API via adding a handler -
> if someone needs
ted scenarios, I have another concern for
> global configuration is that for
> one job, different operators seems to still might have different expected
> behaviors. For example, A
> job using both Window operator and AsyncWaitOperator might have different
> requirements for timers
&g
t,
Yun Gao
--
From:Divye Kapoor
Send Time:2022 Nov. 19 (Sat.) 09:20
To:dev
Subject:Re: Re: [DISCUSS] FLIP-269: Properly Handling the Processing Timers on
Job Termination
Hi Folks,
I raised the initial issue with Yun and others (apologies for the long
email below, I promise
this FLIP?
I'll complete the scenarios in the FLIP.
Great! I am looking forward to understanding more about the use-cases.
Best,
Yun Gao
--
From:Dong Lin
Send Time:2022 Nov. 10 (Thu.) 11:43
To:dev
Cc:Maximilian Michels
Subject:Re:
ock waiting for wallclock time
> to
> > pass instead of stopping the job immediately..
> >
> > - Users have timers to emit message to external systems periodically, and
> > > users want to have one finalize
> > > message at the end of stream.
> >
> >
/jira/browse/FLINK-18647 <
> > https://issues.apache.org/jira/browse/FLINK-18647 >
> > >
> > > wt., 8 lis 2022 o 03:42 Yun Gao
> > napisał(a):
> > >
> > > > Hi Maximilian,
> > > >
> > > > Thanks for the discussion! It
> I'll complete the scenarios in the FLIP.
>
Great! I am looking forward to understanding more about the use-cases.
> Best,
> Yun Gao
> ----------
> From:Dong Lin
> Send Time:2022 Nov. 10 (Thu.) 11:43
> To:dev
e use-case in this FLIP?
I'll complete the scenarios in the FLIP.
Best,
Yun Gao
--
From:Dong Lin
Send Time:2022 Nov. 10 (Thu.) 11:43
To:dev
Cc:Maximilian Michels
Subject:Re: Re: [DISCUSS] FLIP-269: Properly Handling the Processing Timers
c:Maximilian Michels
Subject:Re: Re: [DISCUSS] FLIP-269: Properly Handling the Processing Timers on
Job Termination
Hi Piotr,
I also think the scenario mentioned in this FLIP is useful to address. I am
happy to discuss this together and figure out the more usable APIs.
I guess the choice of API pr
s seems should be dropped
> > instead of
> > triggering. Thus we think it would be necessary to provide per-timer
> > configuration.
> >
> > Best,
> > Yun Gao
> >
> >
> >
> >
> > ------Original Mail ------
> > Sender:Maximilian Michel
; --Original Mail --
> Sender:Maximilian Michels
> Send Date:Fri Nov 4 21:35:58 2022
> Recipients:Flink Dev , Yun Gao >
> Subject:Re: [DISCUSS] FLIP-269: Properly Handling the Processing Timers on
> Job Termination
> Hey Yun,
>
> I wond
:Flink Dev , Yun Gao
Subject:Re: [DISCUSS] FLIP-269: Properly Handling the Processing Timers on Job
Termination
Hey Yun,
I wonder whether we need to add a new option when registering timers. Won't
it be sufficient to flush all pending timers on termination but not allow
new ones to be registered?
-Max
Hi Yun,
Thanks for the proposal.
It appears that the issues mentioned in the motivation section are all
related to using Windows on the DataStream API, where the user's code
typically does not have anything to do with Timer. I am wondering if the
following alternative solution might be more
Hey Yun,
I wonder whether we need to add a new option when registering timers. Won't
it be sufficient to flush all pending timers on termination but not allow
new ones to be registered?
-Max
On Wed, Nov 2, 2022 at 11:20 AM Yun Gao
wrote:
> Hi everyone,
> I would like to open a discussion[1]
Hi everyone,
I would like to open a discussion[1] on how to
properly handle the processing timers on job
termination.
Currently all the processing timers would be
ignored on job termination. This behavior is
not suitable for some cases like WindowOperator.
Thus we'd like to provide more options
28 matches
Mail list logo