It appears that this PR tried to fix things for the Dataflow runner -
https://github.com/apache/beam/pull/11924. It also ensure that if a timer
fired in a bundle but was reset mid bundle to a later time mid bundle, then
we will skip that timer firing.

I believe this bug was also fixed in the direct runner previously. We
probably still need to fix it for other runners and portability.

Reuven

On Fri, Sep 18, 2020 at 4:48 PM Boyuan Zhang <[email protected]> wrote:

> Hi Reuven,
>
> Would you like to share the links to potential fixes? We can figure out
> what we can do there.
>
> On Fri, Sep 18, 2020 at 4:21 PM Reuven Lax <[email protected]> wrote:
>
>>
>>
>> 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.
>>>
>>
>> This "current" behavior is a bug, and one that has led to some weird
>> effects. There have been some PRs attempting to fix it, and I think we
>> should prioritize fixing this bug.
>>
>>
>>> 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