@Override public void processElement(ProcessContext c) throws Exception { KeyedWorkItem<K, InputT> element = c.element(); K key = c.element().key(); TimerInternals timerInternals = c.windowingInternals().timerInternals(); StateInternals<K> stateInternals = stateInternalsFactory.stateInternalsForKey(key); ReduceFnRunner<K, InputT, OutputT, W> reduceFnRunner = new ReduceFnRunner<>( key, windowingStrategy, ExecutableTriggerStateMachine.create( TriggerStateMachines.stateMachineForTrigger(windowingStrategy.getTrigger())), stateInternals, timerInternals, WindowingInternalsAdapters.outputWindowedValue(c.windowingInternals()), WindowingInternalsAdapters.sideInputReader(c.windowingInternals()), droppedDueToClosedWindow, reduceFn, c.getPipelineOptions()); reduceFnRunner.processElements(element.elementsIterable()); for (TimerData timer : element.timersIterable()) { reduceFnRunner.onTimer(timer); } reduceFnRunner.persist(); }
/** * Verifies that the the set of windows that have any state stored is exactly {@code * expectedWindows} and that each of these windows has only tags from {@code allowedTags}. */ private void assertHasOnlyGlobalAndAllowedTags( Set<W> expectedWindows, Set<StateTag<?>> allowedTags) { runner.persist(); Set<StateNamespace> expectedWindowsSet = new HashSet<>(); for (W expectedWindow : expectedWindows) { expectedWindowsSet.add(windowNamespace(expectedWindow)); } Set<StateNamespace> actualWindows = new HashSet<>(); for (StateNamespace namespace : stubContexts.state.getNamespacesInUse()) { if (namespace instanceof StateNamespaces.GlobalNamespace) { continue; } else if (namespace instanceof StateNamespaces.WindowNamespace) { Set<StateTag<?>> tagsInUse = stubContexts.state.getTagsInUse(namespace); if (tagsInUse.isEmpty()) { continue; } actualWindows.add(namespace); Set<StateTag<?>> unexpected = Sets.difference(tagsInUse, allowedTags); if (unexpected.isEmpty()) { continue; } else { fail(namespace + " has unexpected states: " + tagsInUse); } } else if (namespace instanceof StateNamespaces.WindowAndTriggerNamespace) { Set<StateTag<?>> tagsInUse = stubContexts.state.getTagsInUse(namespace); assertTrue(namespace + " contains " + tagsInUse, tagsInUse.isEmpty()); } else { fail("Unrecognized namespace " + namespace); } } assertEquals(expectedWindowsSet, actualWindows); }
public Instant getWatermarkHold() { runner.persist(); return stubContexts.state.minimumWatermarkHold(); }