Personally, I think that it makes sense to not see an old timer after one
has cleared (or moved) it. This is consistent with the idea that the
behavior should be the same if every element was processed in its own
bundle. The fact that a user can (sometimes, but not always) see an old
timer after they set a new one is leaking implementation details.

On Fri, Sep 18, 2020 at 3:14 PM Luke Cwik <[email protected]> wrote:

> PR 12836[1] is adding support for clearing timers and there is a
> discussion about what the semantics for a cleared timer should be.
>
> So far we have:
> 1) Clearing an unset timer is a no-op
> 2) If the last action on the timer was to clear it, then a future bundle
> should not see it fire
>
> Ambiguity occurs if the last action on a timer was to clear it within the
> same bundle then should the current bundle not see it fire if it has yet to
> become visible to the user? Since element processing and timer firings are
> "unordered", this can happen.
>
> Having the clear prevent the timer from firing within the same bundle if
> it has yet to fire could make sense and simplifies clearing timer loops.
> For example:
>
> @ProcessElement
> process(ProcessContext c) {
>   if (initialCondition) {
>     setTimer();
>   } else {
>     clearTimer();
>   }
> }
>
> @OnTimer
> onTimer(...) {
>   do some side effect
>   set timer to fire again in the future
> }
>
> would require logic within the onTimer() method to check to see if we
> should stop instead of relying on the fact that the clear will prevent the
> timer loop.
>
> On the other hand, we currently don't prevent timers from firing that are
> eligible within the same bundle if their firing time is changed within the
> bundle to some future time. Clearing timers could be treated conceptually
> like setting them to "infinity" and hence the current set logic would
> suggest that we shouldn't prevent timer firings that are part of the same
> bundle.
>
> Are there additional use cases that we should consider that suggest one
> approach over the other?
> What do people think?
>
> 1: https://github.com/apache/beam/pull/12836
>

Reply via email to