private static WatermarkUpdate updateAndTrace(String name, Instant oldTime, Instant currentTime) { WatermarkUpdate res = WatermarkUpdate.fromTimestamps(oldTime, currentTime); if (res.isAdvanced()) { WindowTracing.debug("Watermark {} advanced from {} to {}", name, oldTime, currentTime); } return res; }
/** Returns the next eligible processing time timer, if none returns null. */ @Nullable public TimerData removeNextProcessingTimer() { TimerData timer = removeNextTimer(processingTime, TimeDomain.PROCESSING_TIME); if (timer != null) { WindowTracing.trace( "{}.removeNextProcessingTimer: firing {} at {}", getClass().getSimpleName(), timer, processingTime); } return timer; }
/** Returns the next eligible event time timer, if none returns null. */ @Nullable public TimerData removeNextEventTimer() { TimerData timer = removeNextTimer(inputWatermarkTime, TimeDomain.EVENT_TIME); if (timer != null) { WindowTracing.trace( "{}.removeNextEventTimer: firing {} at {}", getClass().getSimpleName(), timer, inputWatermarkTime); } return timer; }
WindowTracing.debug( "{}: Dropping element at {} for key: {}; " + "window: {} since it is too far behind inputWatermark: {}",
/** Returns the next eligible synchronized processing time timer, if none returns null. */ @Nullable public TimerData removeNextSynchronizedProcessingTimer() { TimerData timer = removeNextTimer(synchronizedProcessingTime, TimeDomain.SYNCHRONIZED_PROCESSING_TIME); if (timer != null) { WindowTracing.trace( "{}.removeNextSynchronizedProcessingTimer: firing {} at {}", getClass().getSimpleName(), timer, synchronizedProcessingTime); } return timer; }
WindowTracing.debug( "{}: Dropping element at {} for key: {}; " + "window: {} since it is too far behind inputWatermark: {}",
/** Advances processing time to the given value. */ public void advanceProcessingTime(Instant newProcessingTime) throws Exception { checkNotNull(newProcessingTime); checkState( !newProcessingTime.isBefore(processingTime), "Cannot move processing time backwards from %s to %s", processingTime, newProcessingTime); WindowTracing.trace( "{}.advanceProcessingTime: from {} to {}", getClass().getSimpleName(), processingTime, newProcessingTime); processingTime = newProcessingTime; }
WindowTracing.debug( "{}: Dropping element at {} for key:{}; window:{} " + "since too far behind inputWatermark:{}; outputWatermark:{}",
/** Advances synchronized processing time to the given value. */ public void advanceSynchronizedProcessingTime(Instant newSynchronizedProcessingTime) throws Exception { checkNotNull(newSynchronizedProcessingTime); checkState( !newSynchronizedProcessingTime.isBefore(synchronizedProcessingTime), "Cannot move processing time backwards from %s to %s", synchronizedProcessingTime, newSynchronizedProcessingTime); WindowTracing.trace( "{}.advanceProcessingTime: from {} to {}", getClass().getSimpleName(), synchronizedProcessingTime, newSynchronizedProcessingTime); synchronizedProcessingTime = newSynchronizedProcessingTime; }
@Override public void processElement(WindowedValue<InputT> input) { // StatefulDoFnRunner always observes windows, so we need to explode for (WindowedValue<InputT> value : input.explodeWindows()) { BoundedWindow window = value.getWindows().iterator().next(); if (isLate(window)) { // The element is too late for this window. droppedDueToLateness.inc(); WindowTracing.debug( "StatefulDoFnRunner.processElement: Dropping element at {}; window:{} " + "since too far behind inputWatermark:{}", input.getTimestamp(), window, cleanupTimer.currentInputWatermarkTime()); } else { cleanupTimer.setForWindow(window); doFnRunner.processElement(value); } } }
/** Advances input watermark to the given value. */ public void advanceInputWatermark(Instant newInputWatermark) throws Exception { checkNotNull(newInputWatermark); checkState( !newInputWatermark.isBefore(inputWatermarkTime), "Cannot move input watermark time backwards from %s to %s", inputWatermarkTime, newInputWatermark); WindowTracing.trace( "{}.advanceInputWatermark: from {} to {}", getClass().getSimpleName(), inputWatermarkTime, newInputWatermark); inputWatermarkTime = newInputWatermark; }
/** Advances output watermark to the given value. */ public void advanceOutputWatermark(Instant newOutputWatermark) { checkNotNull(newOutputWatermark); final Instant adjustedOutputWatermark; if (newOutputWatermark.isAfter(inputWatermarkTime)) { WindowTracing.trace( "{}.advanceOutputWatermark: clipping output watermark from {} to {}", getClass().getSimpleName(), newOutputWatermark, inputWatermarkTime); adjustedOutputWatermark = inputWatermarkTime; } else { adjustedOutputWatermark = newOutputWatermark; } checkState( outputWatermarkTime == null || !adjustedOutputWatermark.isBefore(outputWatermarkTime), "Cannot move output watermark time backwards from %s to %s", outputWatermarkTime, adjustedOutputWatermark); WindowTracing.trace( "{}.advanceOutputWatermark: from {} to {}", getClass().getSimpleName(), outputWatermarkTime, adjustedOutputWatermark); outputWatermarkTime = adjustedOutputWatermark; }
@Override public void onTimer( String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { if (cleanupTimer.isForWindow(timerId, window, timestamp, timeDomain)) { stateCleaner.clearForWindow(window); // There should invoke the onWindowExpiration of DoFn } else { // An event-time timer can never be late because we don't allow setting timers after GC time. // Ot can happen that a processing-time time fires for a late window, we need to ignore // this. if (!timeDomain.equals(TimeDomain.EVENT_TIME) && isLate(window)) { // don't increment the dropped counter, only do that for elements WindowTracing.debug( "StatefulDoFnRunner.onTimer: Ignoring processing-time timer at {}; window:{} " + "since window is too far behind inputWatermark:{}", timestamp, window, cleanupTimer.currentInputWatermarkTime()); } else { doFnRunner.onTimer(timerId, window, timestamp, timeDomain); } } }
/** @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. */ @Deprecated @Override public void deleteTimer(TimerData timer) { WindowTracing.trace("{}.deleteTimer: {}", getClass().getSimpleName(), timer); existingTimers.remove(timer.getNamespace(), timer.getTimerId()); timersForDomain(timer.getDomain()).remove(timer); }
/** Clear any remaining holds. */ public void clearHolds(ReduceFn<?, ?, ?, W>.Context context) { WindowTracing.debug( "WatermarkHold.clearHolds: For key:{}; window:{}; inputWatermark:{}; outputWatermark:{}", context.key(), context.window(), timerInternals.currentInputWatermarkTime(), timerInternals.currentOutputWatermarkTime()); context.state().access(elementHoldTag).clear(); context.state().access(EXTRA_HOLD_TAG).clear(); }
/** @deprecated use {@link #setTimer(StateNamespace, String, Instant, TimeDomain)}. */ @Deprecated @Override public void setTimer(TimerData timerData) { WindowTracing.trace("{}.setTimer: {}", getClass().getSimpleName(), timerData); @Nullable TimerData existing = existingTimers.get(timerData.getNamespace(), timerData.getTimerId()); if (existing == null) { existingTimers.put(timerData.getNamespace(), timerData.getTimerId(), timerData); timersForDomain(timerData.getDomain()).add(timerData); } else { checkArgument( timerData.getDomain().equals(existing.getDomain()), "Attempt to set %s for time domain %s, but it is already set for time domain %s", timerData.getTimerId(), timerData.getDomain(), existing.getDomain()); if (!timerData.getTimestamp().equals(existing.getTimestamp())) { NavigableSet<TimerData> timers = timersForDomain(timerData.getDomain()); timers.remove(existing); timers.add(timerData); existingTimers.put(timerData.getNamespace(), timerData.getTimerId(), timerData); } } }
private void cancelEndOfWindowAndGarbageCollectionTimers( ReduceFn<?, ?, ?, W>.Context directContext) { WindowTracing.debug( "ReduceFnRunner.cancelEndOfWindowAndGarbageCollectionTimers: Deleting timers for " + "key:{}; window:{} where inputWatermark:{}; outputWatermark:{}", key, directContext.window(), timerInternals.currentInputWatermarkTime(), timerInternals.currentOutputWatermarkTime()); Instant eow = directContext.window().maxTimestamp(); directContext.timers().deleteTimer(eow, TimeDomain.EVENT_TIME); Instant gc = LateDataUtils.garbageCollectionTime(directContext.window(), windowingStrategy); if (gc.isAfter(eow)) { directContext.timers().deleteTimer(gc, TimeDomain.EVENT_TIME); } }
WindowTracing.trace( "{}.addGarbageCollectionHold: gc hold would be before the input watermark " + "for key:{}; window: {}; inputWatermark: {}; outputWatermark: {}", WindowTracing.trace( "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} is unnecessary " + "since empty pane and FIRE_IF_NON_EMPTY for key:{}; window:{}; inputWatermark:{}; " WindowTracing.trace( "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} is on time for " + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
W window = windowNamespace.getWindow(); WindowTracing.debug( "{}: Received timer key:{}; window:{}; data:{} with " + "inputWatermark:{}; outputWatermark:{}", WindowTracing.debug( "{}: Cleaning up for key:{}; window:{} with inputWatermark:{}; outputWatermark:{}", ReduceFnRunner.class.getSimpleName(), WindowTracing.debug( "{}.onTimers: Triggering for key:{}; window:{} at {} with " + "inputWatermark:{}; outputWatermark:{}", Instant cleanupTime = LateDataUtils.garbageCollectionTime(directContext.window(), windowingStrategy); WindowTracing.debug( "ReduceFnRunner.onTimer: Scheduling cleanup timer for key:{}; window:{} at {} with " + "inputWatermark:{}; outputWatermark:{}",