/** 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; }
/** 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; }
/** 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 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; }
/** 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; }
/** 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; }
/** @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); }
/** @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); } } }
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:{}",
public final void injectElements(List<TimestampedValue<InputT>> values) throws Exception { for (TimestampedValue<InputT> value : values) { WindowTracing.trace("TriggerTester.injectElements: {}", value); } Iterable<WindowedValue<InputT>> inputs = values .stream() .map( input -> { try { InputT value = input.getValue(); Instant timestamp = input.getTimestamp(); Collection<W> windows = windowFn.assignWindows( new TestAssignContext<W>( windowFn, value, timestamp, GlobalWindow.INSTANCE)); return WindowedValue.of(value, timestamp, windows, PaneInfo.NO_FIRING); } catch (Exception e) { throw new RuntimeException(e); } }) .collect(Collectors.toList()); ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner(); runner.processElements( new LateDataDroppingDoFnRunner.LateDataFilter(objectStrategy, timerInternals) .filter(KEY, inputs)); // Persist after each bundle. runner.persist(); }
/** * Advance the input watermark to the specified time, firing any timers that should fire. Then * advance the output watermark as far as possible. */ public void advanceInputWatermark(Instant newInputWatermark) throws Exception { timerInternals.advanceInputWatermark(newInputWatermark); ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner(); while (true) { TimerData timer; List<TimerInternals.TimerData> timers = new ArrayList<>(); while ((timer = timerInternals.removeNextEventTimer()) != null) { timers.add(timer); } if (timers.isEmpty()) { break; } runner.onTimers(timers); } if (autoAdvanceOutputWatermark) { Instant hold = stateInternals.earliestWatermarkHold(); if (hold == null) { WindowTracing.trace( "TestInMemoryTimerInternals.advanceInputWatermark: no holds, " + "so output watermark = input watermark"); hold = timerInternals.currentInputWatermarkTime(); } advanceOutputWatermark(hold); } runner.persist(); }
/** * Schedule a timer to garbage collect the window. * * <p>The timer: * * <ul> * <li>...must be fired strictly after the expiration of the window. * <li>...should be as close to the expiration as possible, to have a timely output of remaining * buffered data, and GC. * </ul> */ private void scheduleGarbageCollectionTimer(ReduceFn<?, ?, ?, W>.Context directContext) { Instant inputWM = timerInternals.currentInputWatermarkTime(); Instant gcTime = LateDataUtils.garbageCollectionTime(directContext.window(), windowingStrategy); WindowTracing.trace( "ReduceFnRunner.scheduleGarbageCollectionTimer: Scheduling at {} for " + "key:{}; window:{} where inputWatermark:{}; outputWatermark:{}", gcTime, key, directContext.window(), inputWM, timerInternals.currentOutputWatermarkTime()); checkState( !gcTime.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE), "Timer %s is beyond end-of-time", gcTime); directContext.timers().setTimer(gcTime, TimeDomain.EVENT_TIME); }
context.state().access(elementHoldTag).add(elementHold); WindowTracing.trace( "WatermarkHold.addHolds: element hold at {} is {} for " + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
public final void injectElements(Collection<TimestampedValue<InputT>> values) throws Exception { for (TimestampedValue<InputT> value : values) { WindowTracing.trace("TriggerTester.injectElements: {}", value);