Mean to also add +Reuven Lax <re...@google.com> On Tue, May 4, 2021 at 9:41 AM Kenneth Knowles <k...@apache.org> wrote:
> Explicitly pinging a couple folks who were involved in the original change > which yours essentially reverts. There's a model question here that I want > to clarify on-list: > > When you have a ParDo setting timers, you have an additional watermark > that must be considered: > > - input element watermark > - output watermark > - *(user) timer watermark* > > The timer watermark is an input to the ParDo. Sometimes you might think of > the "timer channel" as a self loop, where each timer is an element. Each > timer has a timestamp (the output timestamp) and separately some > instructions on when to deliver that timer. This is the same as the usual > difference between event time and processing time. > > The instruction on when to deliver a timer can have two forms: > > - wait a certain amount of processing time > - deliver the timer when the *input element watermark* reaches a time X > > Here is an important point: "cleanup timers" are *not* user timers. They > are an implementation detail. They are not part of the model. The runner's > job is to reclaim resources as windows expire. A user should never be > reasoning about how their timers relate to cleanup timers (except for > resource consumption). Because there is no relationship except that the > cleanup should happen "eventually" and invisibly. > > For a runner author to implement a "cleanup timer" requires a different > mechanism. A window expires when *both* the input element watermark *and* > the timer watermark are past the expiry time. In other words, the cleanup > timer fires according to the minimum of these watermarks, combined. It > *cannot* fire according to the input element watermark. If you naively try > to implement it as a user timer, it will be incorrect. Incidentally this is > why @OnWindowExpiration is a meaningful feature. > > Kenn > > On Tue, May 4, 2021 at 4:29 AM Jan Lukavský <je...@seznam.cz> wrote: > >> Hi Kenn, >> >> I created BEAM-12276 [1] with PR [2]. >> >> Jan >> >> [1] https://issues.apache.org/jira/browse/BEAM-12276 >> >> [2] https://github.com/apache/beam/pull/14718 >> On 5/3/21 7:46 PM, Kenneth Knowles wrote: >> >> This seems like just a bug. If you set a timer for X and have output >> timestamp Y where X < Y this should be fine. Is the problem the current >> input watermark? Are you trying to set a timer with output timestamp that >> is already past? I think that should be allowed, too, as long as the window >> is not expired, but I may be missing something. >> >> Some greater detail would be useful - maybe the full stack trace and/or a >> failing unit test in a PR? >> >> Kenn >> >> On Thu, Apr 29, 2021 at 12:51 AM Jan Lukavský <je...@seznam.cz> wrote: >> >>> Hi, >>> >>> I have come across a bug with timer output timestamp - when using event >>> time and relative timers, setting the timer can arbitrarily throw >>> IllegalArgumentException if the firing timestamp (input watermark) is >>> ahead of the output timestamp (like .java.lang.IllegalArgumentException: >>> Attempted to set an event-time timer with an output timestamp of >>> 2021-04-29T07:16:19.369Z that is after the timer firing timestamp >>> -290308-12-21T19:59:05.225Z). But there is no way to access the firing >>> timestamp from user code. This means that the use has to either catch >>> the IllegalArgumentException, or not use this construct at all. >>> >>> Catching the exception should probably not be part of a contract, so we >>> should do one of the following: >>> >>> a) either throw the exception right away and disable using relative >>> timers with output timestamp completely, or >>> >>> b) support it correctly >>> >>> What is the actual reason not to support output timestamps, that are >>> ahead of firing timestamp? From my understanding, that should not be an >>> issue, because there is TimestampCombiner.EARLIEST on the >>> watermarkholdstate that corresponds to the output timestamp. If that is >>> correct can we simply remove the check? >>> >>> Jan >>> >>>