I initially thought that the order of the watermarks could be altered
because of the keying, but I eventually read somewhere that watermarks are
broadcast downstream, so I think what I was thinking isn't relevant.

Regards,
Alexis.

Am Do., 6. Nov. 2025 um 15:27 Uhr schrieb David Anderson <
[email protected]>:

> It is customary to do the watermarking as early as possible in the
> pipeline, and since FLIP-27 best practice has been to have the sources
> apply the watermark strategy. As the documentation [1] points out:
>
> > There are two places in Flink applications where a WatermarkStrategy can
> be used: 1) directly on sources and 2) after non-source operation. The
> first option is preferable, because it allows sources to exploit knowledge
> about shards/partitions/splits in the watermarking logic. Sources can
> usually then track watermarks at a finer level and the overall watermark
> produced by a source will be more accurate.
>
> Why are you thinking it would be preferable to do the keyed partitioning
> first?
>
> That said, I believe what you have proposed will work.
>
> Best,
> David
>
> [1]
> https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/datastream/event-time/generating_watermarks/#using-watermark-strategies
>
>
> On Tue, Oct 28, 2025 at 1:38 AM Alexis Sarda-Espinosa <
> [email protected]> wrote:
>
>> Hello,
>>
>> I'm investigating switching our windowing to event time, and I figured it
>> would be good to assign timestamps/watermarks *after* the keyBy we need
>> for the windows. However, I see the keyBy method instantiates a
>> PartitionTransformation whose Javadoc states: "This does not create a
>> physical operation, it only affects how upstream operations are connected
>> to downstream operations."
>>
>> So the question is, if I have a DataStream and I define the job like this:
>>
>> var watermarked = stream.keyBy(...)
>>     .assignTimestampsAndWatermarks(...)
>>
>> DataStreamUtils.reinterpretAsKeyedStream(watermarked, ...)
>>     .window(...)
>>
>> Is this actually working as expected?
>>
>> Regards,
>> Alexis.
>>
>

Reply via email to