[
https://issues.apache.org/jira/browse/BEAM-7520?focusedWorklogId=298855&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-298855
]
ASF GitHub Bot logged work on BEAM-7520:
----------------------------------------
Author: ASF GitHub Bot
Created on: 21/Aug/19 16:56
Start Date: 21/Aug/19 16:56
Worklog Time Spent: 10m
Work Description: je-ik commented on pull request #9190: [BEAM-7520] Fix
timer firing order in DirectRunner
URL: https://github.com/apache/beam/pull/9190#discussion_r316293694
##########
File path:
runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
##########
@@ -244,27 +250,40 @@ public void
processElement(WindowedValue<KeyedWorkItem<K, KV<K, InputT>>> gbkRes
delegateEvaluator.processElement(windowedValue);
}
+ @Nullable 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);
Review comment:
Generally yes. The problem is that you cannot fire multiple timers at once,
because the outcome of firing a single timer is unknown. It might or might not
setup some other timer. If it was to be solved, it seems to me it would require
`PushbackSideInputDoFnRunner` (and most likely `DoFnRunner` to return set
timers, instead of `void`. The other option would be somehow hack into
`WatermarkManager` and be able to retrieve "most recent timers set", os
something like that (which seems to be fragile and hackish). The correct
solution seems to be a too big refactor, given that the performance penalty for
the solution proposed in this PR seems to be low.
----------------------------------------------------------------
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: 298855)
Time Spent: 8h (was: 7h 50m)
> 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: 8h
> 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)