Hi Maro,
Do you wanna to do a cascade Tumbling windowed stream Job? Do you hope the
data generated in the first Tumbling windowed stream could belongs to same
window in the second stream?
We could understand your demands better if you could share the code.

I would try to explain some detailed problems in your email. Hope it helps.
> when the watermark is GREATER than 00:45:00.000, which means that the
event time created by a timer that fires at 00:45:00.000
If first windowed stream receives watermark 00:45:00.000, it means there
should be no more elements from the input stream with a timestamp which is
older or equals to 00:45:00.000. So it's safe to trigger event timer
00:45:00.000. [1]
However, we need register a timer 00:44:59.999 instead of 00:45:00.000
 because 00:44:59.999 represents the inclusive upper of the window . This
is also the behavior in Window Operator which registers a window trigger
timer (00:44:59.999) for window [00:30:00, 00:45:00) .
Window end is exclusive upper bound of the trigged window while window max
timestamp is inclusive upper bound of the trigged window.

[1]
https://ci.apache.org/projects/flink/flink-docs-master/docs/concepts/time/#event-time-and-watermarks

Best,
JING ZHANG

Marco Villalobos <mvillalo...@kineteque.com> 于2021年9月7日周二 下午11:45写道:

> I am guessing the answer is right infront of me.
>
> If the window has these attributes:
>
> window start: 00:30:00.000.
> window end: 00:45:00.000
> max timestamp: 00:44:59.999
>
> Then perhaps I need the timer to fire at 00:44:59.999 if I want an element
> that it generates too belong to this window.
>
> I'll test my hypothesis later today.
>
>
> On Sep 7, 2021, at 2:07 AM, JING ZHANG <beyond1...@gmail.com> wrote:
>
> Hi Marco,
> I'm not sure which API or SQL query do you use.
> If you use Windowed Stream API in DataStream [1]. The input data would be
> assigned to a Window based on which Window Assigner do you use. If using
> tumbling window, the input data would be assigned to a Window based on it's
> event-time and registers an event timer trigger for the Window. You could
> find more information in document [1]. When watermark passed the end of
> Window, the window would be triggered. The event time of the output element
> is inclusive upper bound of the trigged window.
> If you use Window Aggregation based on Flink SQL [2]. The situation is
> very similar.
>
> [1]
> https://ci.apache.org/projects/flink/flink-docs-master/docs/dev/datastream/operators/windows/
> [2]
> https://ci.apache.org/projects/flink/flink-docs-master/docs/dev/table/sql/queries/window-agg/
>
> Best regards,
> JING ZHANG
>
> Marco Villalobos <mvillalo...@kineteque.com> 于2021年9月7日周二 下午2:24写道:
>
>> If an event time timer is registered to fire exactly every 15 minutes,
>> starting from exactly at the top of the hour (exactly 00:00, 00:15, 00:30,
>> 00:45 for example), and within that timer it produces an element in the
>> stream, what event time will that element have, and what window will it
>> belong to if I am using a 15 minute tumbling event time window?
>>
>
>

Reply via email to