New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[BEAM-8543] Dataflow streaming timers are not strictly time ordered when set earlier mid-bundle #11924
[BEAM-8543] Dataflow streaming timers are not strictly time ordered when set earlier mid-bundle #11924
Changes from 2 commits
9c1dd6a
b73c770
fba4c3d
032bde5
abd93ec
dc7b614
10487f0
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,10 +24,12 @@ | |
import com.google.api.services.dataflow.model.SideInputInfo; | ||
import java.io.Closeable; | ||
import java.io.IOException; | ||
import java.util.Comparator; | ||
import java.util.HashMap; | ||
import java.util.Iterator; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.PriorityQueue; | ||
import java.util.Set; | ||
import java.util.concurrent.ThreadLocalRandom; | ||
import java.util.concurrent.atomic.AtomicLong; | ||
|
@@ -522,6 +524,7 @@ public void start( | |
|
||
this.cachedFiredTimers = null; | ||
this.cachedFiredUserTimers = null; | ||
this.toBeFiredTimersOrdered.clear(); | ||
} | ||
|
||
public void flushState() { | ||
|
@@ -564,10 +567,13 @@ public <W extends BoundedWindow> TimerData getNextFiredTimer(Coder<W> windowCode | |
// Lazily initialized | ||
private Iterator<TimerData> cachedFiredUserTimers = null; | ||
|
||
private PriorityQueue<TimerData> toBeFiredTimersOrdered = | ||
new PriorityQueue<>(Comparator.comparing(TimerData::getTimestamp)); | ||
|
||
public <W extends BoundedWindow> TimerData getNextFiredUserTimer(Coder<W> windowCoder) { | ||
if (cachedFiredUserTimers == null) { | ||
cachedFiredUserTimers = | ||
FluentIterable.<Timer>from(StreamingModeExecutionContext.this.getFiredTimers()) | ||
FluentIterable.from(StreamingModeExecutionContext.this.getFiredTimers()) | ||
.filter( | ||
timer -> | ||
WindmillTimerInternals.isUserTimer(timer) | ||
|
@@ -577,12 +583,21 @@ public <W extends BoundedWindow> TimerData getNextFiredUserTimer(Coder<W> window | |
WindmillTimerInternals.windmillTimerToTimerData( | ||
WindmillNamespacePrefix.USER_NAMESPACE_PREFIX, timer, windowCoder)) | ||
.iterator(); | ||
|
||
cachedFiredUserTimers.forEachRemaining(toBeFiredTimersOrdered::add); | ||
} | ||
|
||
Instant currentInputWatermark = userTimerInternals.currentInputWatermarkTime(); | ||
if (userTimerInternals.hasTimerBefore(currentInputWatermark)) { | ||
while (!toBeFiredTimersOrdered.isEmpty()) { | ||
userTimerInternals.setTimer(toBeFiredTimersOrdered.poll()); | ||
} | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @kennknowles for comment. This doesn't look right to me, as I don't think we should be modifying the WindmillTimerInternals here. I think we just want to merge the timer modifications from processing the workitem into this priority queue; note that if timers are deleted, we need to detect that as well and remove from the priority queue. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yea I don't actually understand what this block is for. FWIW to do timer deletion/reset cheaply without building a bespoke data structure just keep a map from id to firing time or tombstone. This way, whenever a timer comes up in the prio queue you pull out the actual time for it from the map. If it is actually set for another time, don't fire it. If it is obsolete, don't fire it. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @reuvenlax done |
||
|
||
if (!cachedFiredUserTimers.hasNext()) { | ||
if (toBeFiredTimersOrdered.isEmpty()) { | ||
return null; | ||
} | ||
TimerData nextTimer = cachedFiredUserTimers.next(); | ||
TimerData nextTimer = toBeFiredTimersOrdered.poll(); | ||
// User timers must be explicitly deleted when delivered, to release the implied hold | ||
userTimerInternals.deleteTimer(nextTimer); | ||
return nextTimer; | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we even need
cachedFiredUserTimers
? It seems obsolete if we populate the priority queue. The name is also wrong - even before this PR it wasn't a cache. It is a lazily initialized iterator. Instead, we should have a lazily initialized priority queue (like you do) and just a flag to say whether the incoming timers have been loaded yet.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
done