[
https://issues.apache.org/jira/browse/BEAM-7520?focusedWorklogId=298801&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-298801
]
ASF GitHub Bot logged work on BEAM-7520:
----------------------------------------
Author: ASF GitHub Bot
Created on: 21/Aug/19 15:45
Start Date: 21/Aug/19 15:45
Worklog Time Spent: 10m
Work Description: kennknowles commented on pull request #9190:
[BEAM-7520] Fix timer firing order in DirectRunner
URL: https://github.com/apache/beam/pull/9190#discussion_r316260617
##########
File path:
runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
##########
@@ -244,27 +249,40 @@ public void
processElement(WindowedValue<KeyedWorkItem<K, KV<K, InputT>>> gbkRes
delegateEvaluator.processElement(windowedValue);
}
+ Instant lastFired = null;
for (TimerData timer : gbkResult.getValue().timersIterable()) {
checkState(
timer.getNamespace() instanceof WindowNamespace,
"Expected Timer %s to be in a %s, but got %s",
timer,
WindowNamespace.class.getSimpleName(),
timer.getNamespace().getClass().getName());
- WindowNamespace<?> windowNamespace = (WindowNamespace)
timer.getNamespace();
- BoundedWindow timerWindow = windowNamespace.getWindow();
- delegateEvaluator.onTimer(timer, timerWindow);
+ checkState(
+ lastFired == null || !lastFired.isAfter(timer.getTimestamp()),
+ "lastFired was %s, current %s",
+ lastFired,
+ timer.getTimestamp());
+ if (lastFired != null && lastFired.isBefore(timer.getTimestamp())) {
+ pushedBackTimers.add(timer);
+ } else {
+ lastFired = timer.getTimestamp();
+ WindowNamespace<?> windowNamespace = (WindowNamespace)
timer.getNamespace();
+ BoundedWindow timerWindow = windowNamespace.getWindow();
+ delegateEvaluator.onTimer(timer, timerWindow);
Review comment:
I don't think it is quite what I might. Here:
t1.set(50)
t2.set(60)
watermark advances to 100
t1 fires and sets t2 to 40 -- since it is moved earlier it will fire in the
same bundle
t2 fires and sets t1 to 30 -- will not fire in the same bundle, but will be
committed in the result?
I guess this might get the right result... I was just expecting all the
timers that could fire to do it in the same bundle. It is a bit weird that the
normal case of timers just coming in order and not setting anything will not
fire in one bundle. Did I read that code wrong? This is why I suggested a prio
queue, to remove all the logic about `lastFired` and commits. You just fire all
the timers, in order (including newly set timers), until the bundle is done. I
wonder if there is a use case for always setting a timer for the same time and
hoping the bundle will still commit over and over again.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 298801)
Time Spent: 7.5h (was: 7h 20m)
> DirectRunner timers are not strictly time ordered
> -------------------------------------------------
>
> Key: BEAM-7520
> URL: https://issues.apache.org/jira/browse/BEAM-7520
> Project: Beam
> Issue Type: Bug
> Components: runner-direct
> Affects Versions: 2.13.0
> Reporter: Jan Lukavský
> Assignee: Jan Lukavský
> Priority: Major
> Time Spent: 7.5h
> Remaining Estimate: 0h
>
> Let's suppose we have the following situation:
> - statful ParDo with two timers - timerA and timerB
> - timerA is set for window.maxTimestamp() + 1
> - timerB is set anywhere between <windowStart, windowEnd), let's denote that
> timerB.timestamp
> - input watermark moves to BoundedWindow.TIMESTAMP_MAX_VALUE
> Then the order of timers is as follows (correct):
> - timerB
> - timerA
> But, if timerB sets another timer (say for timerB.timestamp + 1), then the
> order of timers will be:
> - timerB (timerB.timestamp)
> - timerA (BoundedWindow.TIMESTAMP_MAX_VALUE)
> - timerB (timerB.timestamp + 1)
> Which is not ordered by timestamp. The reason for this is that when the input
> watermark update is evaluated, the WatermarkManager,extractFiredTimers() will
> produce both timerA and timerB. That would be correct, but when timerB sets
> another timer, that breaks this.
--
This message was sent by Atlassian Jira
(v8.3.2#803003)