Ah yes, Kenn is correct, and i forget we made that change.

To clarify - Beam does not expose late elements as a concept, rather it
exposes late panes on its triggering API. The reason we made the change was
not just because we wanted to include as much data as possible. but also
because we wanted to preserve reasonable invariants around late data (e.g.
one invariant should be that the result of processing an on-time pane
should not produce late data). Simply checking element timestamps against
the watermark is non deterministic (since the watermark advances
asynchronously), so we moved to a different definition of late data.

Reuven

On Fri, Jan 18, 2019 at 11:24 AM Kenneth Knowles <k...@google.com> wrote:

> That is correct. For global window there is no such thing as late data.
>
> Kenn
>
> On Fri, Jan 18, 2019, 11:13 Ruoyun Huang <ruo...@google.com wrote:
>
>> Very helpful discussion (and the fixing PR).
>>
>> To make sure my take-way is correct. The status quo is a) "for a Global
>> Window, then there is *no possible scenario* where data is identified as
>> late".  Rather than b) "for a global window we *no longer* compare
>> watermark to identify late data, but *there is still other criteria*
>> that determines data late".
>>
>> a) is correct and b) is not.  Is that so?
>>
>> On Thu, Jan 17, 2019 at 8:57 PM Kenneth Knowles <k...@google.com> wrote:
>>
>>> Actually, Reuven, that's no longer the case.
>>>
>>> It used to be that incoming data was compared to the watermark but it is
>>> not today. Instead, Jeff's first phrasing is perfect.
>>>
>>> One way to see it is the think about what are the consequences of late
>>> data: if there is a grouping/aggregation by key+window, the window
>>> determines when the grouping is complete. We go ahead and include any data
>>> that shows up before the window is complete. And if you set up allowed
>>> lateness it matches exactly: any data that arrives before the ON_TIME
>>> output gets to be in that output.
>>>
>>> Previously, when we compared incoming elements to the watermark
>>> directly, you could have a window that was still being aggregated but the
>>> elements that fell in the window were dropped. There was no technical
>>> benefit to losing this data, so we stopped dropping it. We also had lots of
>>> tricky bugs and hard-to-manage code related to what we do if an element
>>> arrives after the watermark. And you could have an ON_TIME firing that
>>> included a bunch of "late" data which is confusing.
>>>
>>> Now it is simple: if the window is still alive, the element goes into it.
>>>
>>> I very rarely use the term "late data" when describing Beam's semantics
>>> anyhow. I always found the term / definition a bit arbitrary.
>>>
>>> Kenn
>>>
>>> On Thu, Jan 17, 2019 at 8:13 PM Rui Wang <ruw...@google.com> wrote:
>>>
>>>> I created this PR: https://github.com/apache/beam/pull/7556
>>>>
>>>> Feel free to review/comment it.
>>>>
>>>> -Rui
>>>>
>>>> On Thu, Jan 17, 2019 at 2:37 PM Rui Wang <ruw...@google.com> wrote:
>>>>
>>>>> It might be better to keep something like "watermark usually
>>>>> consistently moves forward". But "Elements that arrive with a smaller
>>>>> timestamp than the current watermark are considered late data." has 
>>>>> already
>>>>> given the order of late data ts and watermark.
>>>>>
>>>>>
>>>>> -Rui
>>>>>
>>>>> On Thu, Jan 17, 2019 at 1:39 PM Jeff Klukas <jklu...@mozilla.com>
>>>>> wrote:
>>>>>
>>>>>> Reuven - I don't think I realized it was possible to have late data
>>>>>> with the global window, so I'm definitely learning things through this
>>>>>> discussion.
>>>>>>
>>>>>> New suggested wording, then:
>>>>>>
>>>>>>     Elements that arrive with a smaller timestamp than the current
>>>>>> watermark are considered late data.
>>>>>>
>>>>>> That says basically the same thing as the wording currently in the
>>>>>> guide, but uses "smaller" (which implies a less-than-watermark 
>>>>>> comparison)
>>>>>> rather than "later" (which folks have interpreted as a
>>>>>> greater-than-watermark comparison).
>>>>>>
>>>>>> On Thu, Jan 17, 2019 at 3:40 PM Reuven Lax <re...@google.com> wrote:
>>>>>>
>>>>>>> Though it's not tied to window. You could be in the global window,
>>>>>>> so the watermark never advances past the end of the window, yet still 
>>>>>>> get
>>>>>>> late data.
>>>>>>>
>>>>>>> On Thu, Jan 17, 2019, 11:14 AM Jeff Klukas <jklu...@mozilla.com
>>>>>>> wrote:
>>>>>>>
>>>>>>>> How about: "Once the watermark progresses past the end of a window,
>>>>>>>> any further elements that arrive with a timestamp in that window are
>>>>>>>> considered late data."
>>>>>>>>
>>>>>>>> On Thu, Jan 17, 2019 at 1:43 PM Rui Wang <ruw...@google.com> wrote:
>>>>>>>>
>>>>>>>>> Hi Community,
>>>>>>>>>
>>>>>>>>> In Beam programming guide [1], there is a sentence: "Data that
>>>>>>>>> arrives with a timestamp after the watermark is considered *late
>>>>>>>>> data*"
>>>>>>>>>
>>>>>>>>> Seems like people get confused by it. For example, see
>>>>>>>>> Stackoverflow comment [2]. Basically it makes people think that a 
>>>>>>>>> event
>>>>>>>>> timestamp that is bigger than watermark is considered late (due to 
>>>>>>>>> that
>>>>>>>>> "after").
>>>>>>>>>
>>>>>>>>> Although there is a example right after this sentence to explain
>>>>>>>>> late data, seems to me that this sentence is incomplete. The complete
>>>>>>>>> sentence to me can be: "The watermark consistently advances from -inf 
>>>>>>>>> to
>>>>>>>>> +inf. Data that arrives with a timestamp after the watermark is 
>>>>>>>>> considered
>>>>>>>>> late data."
>>>>>>>>>
>>>>>>>>> Am I understand correctly? Is there better description for the
>>>>>>>>> order of late data and watermark? I would happy to send PR to update 
>>>>>>>>> Beam
>>>>>>>>> documentation.
>>>>>>>>>
>>>>>>>>> -Rui
>>>>>>>>>
>>>>>>>>> [1]:
>>>>>>>>> https://beam.apache.org/documentation/programming-guide/#windowing
>>>>>>>>> [2]:
>>>>>>>>> https://stackoverflow.com/questions/54141352/dataflow-to-process-late-and-out-of-order-data-for-batch-and-stream-messages/54188971?noredirect=1#comment95302476_54188971
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>
>> --
>> ================
>> Ruoyun  Huang
>>
>>

Reply via email to