@Override public boolean isReady(PCollectionView<?> sideInput, BoundedWindow window) { Set<BoundedWindow> readyWindows = stateInternals.state(StateNamespaces.global(), availableWindowsTags.get(sideInput)).read(); boolean result = readyWindows != null && readyWindows.contains(window); return result; }
BufferedOutputManager( Output<StreamRecord<WindowedValue<OutputT>>> output, TupleTag<OutputT> mainTag, Map<TupleTag<?>, OutputTag<WindowedValue<?>>> tagsToOutputTags, final Map<TupleTag<?>, Coder<WindowedValue<?>>> tagsToCoders, Map<TupleTag<?>, Integer> tagsToIds, StateInternals stateInternals) { this.output = output; this.mainTag = mainTag; this.tagsToOutputTags = tagsToOutputTags; this.tagsToIds = tagsToIds; this.idsToTags = new HashMap<>(); for (Map.Entry<TupleTag<?>, Integer> entry : tagsToIds.entrySet()) { idsToTags.put(entry.getValue(), entry.getKey()); } ImmutableMap.Builder<Integer, Coder<WindowedValue<?>>> idsToCodersBuilder = ImmutableMap.builder(); for (Map.Entry<TupleTag<?>, Integer> entry : tagsToIds.entrySet()) { idsToCodersBuilder.put(entry.getValue(), tagsToCoders.get(entry.getKey())); } StateTag<BagState<KV<Integer, WindowedValue<?>>>> bufferTag = StateTags.bag("bundle-buffer-tag", new TaggedKvCoder(idsToCodersBuilder.build())); bufferState = stateInternals.state(StateNamespaces.global(), bufferTag); }
private void checkInitPushedBackWatermark() { // init and restore from pushedBack state. // Not done in initializeState, because OperatorState is not ready. if (!pushedBackWatermark.isPresent()) { BagState<WindowedValue<InputT>> pushedBack = nonKeyedStateInternals.state(StateNamespaces.global(), pushedBackTag); long min = Long.MAX_VALUE; for (WindowedValue<InputT> value : pushedBack.read()) { min = Math.min(min, value.getTimestamp().getMillis()); } setPushedBackWatermark(min); } }
@Override public void close() throws Exception { super.close(); // sanity check: these should have been flushed out by +Inf watermarks if (!sideInputs.isEmpty() && nonKeyedStateInternals != null) { BagState<WindowedValue<InputT>> pushedBack = nonKeyedStateInternals.state(StateNamespaces.global(), pushedBackTag); Iterable<WindowedValue<InputT>> pushedBackContents = pushedBack.read(); if (pushedBackContents != null) { if (!Iterables.isEmpty(pushedBackContents)) { String pushedBackString = Joiner.on(",").join(pushedBackContents); throw new RuntimeException( "Leftover pushed-back data: " + pushedBackString + ". This indicates a bug."); } } } checkFinishBundleTimer.cancel(true); doFnInvoker.invokeTeardown(); }
@Test public void testFromStringGlobal() { assertStringKeyRoundTrips(intervalCoder, StateNamespaces.global()); }
/** * Add the given value to the internal side-input store of the given side input. This might change * the result of {@link #isReady(PCollectionView, BoundedWindow)} for that side input. */ public void addSideInputValue(PCollectionView<?> sideInput, WindowedValue<Iterable<?>> value) { @SuppressWarnings("unchecked") Coder<BoundedWindow> windowCoder = (Coder<BoundedWindow>) sideInput.getWindowingStrategyInternal().getWindowFn().windowCoder(); StateTag<ValueState<Iterable<?>>> stateTag = sideInputContentsTags.get(sideInput); for (BoundedWindow window : value.getWindows()) { stateInternals .state(StateNamespaces.window(windowCoder, window), stateTag) .write(value.getValue()); stateInternals .state(StateNamespaces.global(), availableWindowsTags.get(sideInput)) .add(window); } }
@Override public final void processElement1( StreamRecord<WindowedValue<InputT>> streamRecord) throws Exception { checkInvokeStartBundle(); Iterable<WindowedValue<InputT>> justPushedBack = pushbackDoFnRunner.processElementInReadyWindows(streamRecord.getValue()); BagState<WindowedValue<InputT>> pushedBack = nonKeyedStateInternals.state(StateNamespaces.global(), pushedBackTag); checkInitPushedBackWatermark(); long min = pushedBackWatermark.get(); for (WindowedValue<InputT> pushedBackValue : justPushedBack) { min = Math.min(min, pushedBackValue.getTimestamp().getMillis()); pushedBack.add(pushedBackValue); } setPushedBackWatermark(min); checkInvokeFinishBundleByCount(); }
/** * Emits all pushed-back data. This should be used once we know that there will not be * any future side input, i.e. that there is no point in waiting. */ private void emitAllPushedBackData() throws Exception { BagState<WindowedValue<InputT>> pushedBack = nonKeyedStateInternals.state(StateNamespaces.global(), pushedBackTag); Iterable<WindowedValue<InputT>> pushedBackContents = pushedBack.read(); if (pushedBackContents != null) { for (WindowedValue<InputT> elem : pushedBackContents) { // we need to set the correct key in case the operator is // a (keyed) window operator setKeyContextElement1(new StreamRecord<>(elem)); doFnRunner.processElement(elem); } } pushedBack.clear(); setPushedBackWatermark(Long.MAX_VALUE); }
public MergingActiveWindowSet(WindowFn<Object, W> windowFn, StateInternals state) { this.windowFn = windowFn; StateTag<ValueState<Map<W, Set<W>>>> tag = StateTags.makeSystemTagInternal( StateTags.value( "tree", MapCoder.of(windowFn.windowCoder(), SetCoder.of(windowFn.windowCoder())))); valueState = state.state(StateNamespaces.global(), tag); // Little use trying to prefetch this state since the ReduceFnRunner // is stymied until it is available. activeWindowToStateAddressWindows = emptyIfNull(valueState.read()); originalActiveWindowToStateAddressWindows = deepCopy(activeWindowToStateAddressWindows); }
@Test public void testFromStringGlobalWindow() { assertStringKeyRoundTrips(GlobalWindow.Coder.INSTANCE, StateNamespaces.global()); assertStringKeyRoundTrips( GlobalWindow.Coder.INSTANCE, StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE)); assertStringKeyRoundTrips( GlobalWindow.Coder.INSTANCE, StateNamespaces.windowAndTrigger(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE, 18)); }
@Test public void testCompareEqual() { Instant timestamp = new Instant(100); StateNamespace namespace = StateNamespaces.global(); TimerData timer = TimerData.of("id", namespace, timestamp, TimeDomain.EVENT_TIME); assertThat( timer, comparesEqualTo(TimerData.of("id", namespace, timestamp, TimeDomain.EVENT_TIME))); }
@Test public void testCompareByTimerId() { Instant timestamp = new Instant(100); StateNamespace namespace = StateNamespaces.global(); TimerData id0Timer = TimerData.of("id0", namespace, timestamp, TimeDomain.EVENT_TIME); TimerData id1Timer = TimerData.of("id1", namespace, timestamp, TimeDomain.EVENT_TIME); assertThat(id0Timer, lessThan(id1Timer)); } }
@Test public void testCompareByTimestamp() { Instant firstTimestamp = new Instant(100); Instant secondTimestamp = new Instant(200); StateNamespace namespace = StateNamespaces.global(); TimerData firstTimer = TimerData.of(namespace, firstTimestamp, TimeDomain.EVENT_TIME); TimerData secondTimer = TimerData.of(namespace, secondTimestamp, TimeDomain.EVENT_TIME); assertThat(firstTimer, lessThan(secondTimer)); }
/** Test that WindowAndTrigger namespaces are prefixed by the related Window namespace. */ @Test public void testGlobalWindowPrefixing() { StateNamespace window = StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE); StateNamespace windowAndTrigger = StateNamespaces.windowAndTrigger(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE, 57); assertThat(windowAndTrigger.stringKey(), Matchers.startsWith(window.stringKey())); assertThat( StateNamespaces.global().stringKey(), Matchers.not(Matchers.startsWith(window.stringKey()))); }
/** Test that WindowAndTrigger namespaces are prefixed by the related Window namespace. */ @Test public void testIntervalWindowPrefixing() { StateNamespace window = StateNamespaces.window(intervalCoder, intervalWindow(1000, 87392)); StateNamespace windowAndTrigger = StateNamespaces.windowAndTrigger(intervalCoder, intervalWindow(1000, 87392), 57); assertThat(windowAndTrigger.stringKey(), Matchers.startsWith(window.stringKey())); assertThat( StateNamespaces.global().stringKey(), Matchers.not(Matchers.startsWith(window.stringKey()))); }
@Test public void testCompareByDomain() { Instant timestamp = new Instant(100); StateNamespace namespace = StateNamespaces.global(); TimerData eventTimer = TimerData.of(namespace, timestamp, TimeDomain.EVENT_TIME); TimerData procTimer = TimerData.of(namespace, timestamp, TimeDomain.PROCESSING_TIME); TimerData synchronizedProcTimer = TimerData.of(namespace, timestamp, TimeDomain.SYNCHRONIZED_PROCESSING_TIME); assertThat(eventTimer, lessThan(procTimer)); assertThat(eventTimer, lessThan(synchronizedProcTimer)); assertThat(procTimer, lessThan(synchronizedProcTimer)); }
@Test public void testSerialization() { TimerDataCoder timerDataCoder = TimerDataCoder.of(GlobalWindow.Coder.INSTANCE); TimerData timerData = TimerData.of( "arbitrary-id", StateNamespaces.global(), new Instant(0), TimeDomain.EVENT_TIME); String key = "key"; ApexTimerInternals<String> timerInternals = new ApexTimerInternals<>(timerDataCoder); timerInternals.setContext(key, StringUtf8Coder.of(), Instant.now(), null); timerInternals.setTimer(timerData); ApexTimerInternals<String> cloned; assertNotNull("Serialization", cloned = KryoCloneUtils.cloneObject(timerInternals)); cloned.setContext(key, StringUtf8Coder.of(), Instant.now(), null); Map<?, Set<Slice>> timers = cloned.getTimerSet(TimeDomain.EVENT_TIME).getMap(); assertEquals(1, timers.size()); } }
@Test public void testTimerDataCoder() throws Exception { CoderProperties.coderDecodeEncodeEqual( TimerDataCoder.of(GlobalWindow.Coder.INSTANCE), TimerData.of( "arbitrary-id", StateNamespaces.global(), new Instant(0), TimeDomain.EVENT_TIME)); Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder(); CoderProperties.coderDecodeEncodeEqual( TimerDataCoder.of(windowCoder), TimerData.of( "another-id", StateNamespaces.window( windowCoder, new IntervalWindow(new Instant(0), new Instant(100))), new Instant(99), TimeDomain.PROCESSING_TIME)); }
@Test public void testEncodeDecodeEqual() throws Exception { Iterable<TimerData> timers = ImmutableList.of( TimerData.of(StateNamespaces.global(), new Instant(500L), TimeDomain.EVENT_TIME)); Iterable<WindowedValue<Integer>> elements = ImmutableList.of( WindowedValue.valueInGlobalWindow(1), WindowedValue.valueInGlobalWindow(4), WindowedValue.valueInGlobalWindow(8)); KeyedWorkItemCoder<String, Integer> coder = KeyedWorkItemCoder.of(StringUtf8Coder.of(), VarIntCoder.of(), GlobalWindow.Coder.INSTANCE); CoderProperties.coderDecodeEncodeEqual(coder, KeyedWorkItems.workItem("foo", timers, elements)); CoderProperties.coderDecodeEncodeEqual(coder, KeyedWorkItems.elementsWorkItem("foo", elements)); CoderProperties.coderDecodeEncodeEqual(coder, KeyedWorkItems.timersWorkItem("foo", timers)); }
/** * This test should not be changed. It verifies that the stringKey matches certain expectations. * If this changes, the ability to reload any pipeline that has persisted these namespaces will be * impacted. */ @Test public void testStability() { StateNamespace global = StateNamespaces.global(); StateNamespace intervalWindow = StateNamespaces.window(intervalCoder, intervalWindow(1000, 87392)); StateNamespace intervalWindowAndTrigger = StateNamespaces.windowAndTrigger(intervalCoder, intervalWindow(1000, 87392), 57); StateNamespace globalWindow = StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE); StateNamespace globalWindowAndTrigger = StateNamespaces.windowAndTrigger(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE, 12); assertEquals("/", global.stringKey()); assertEquals("/gAAAAAABVWD4ogU/", intervalWindow.stringKey()); assertEquals("/gAAAAAABVWD4ogU/1L/", intervalWindowAndTrigger.stringKey()); assertEquals("//", globalWindow.stringKey()); assertEquals("//C/", globalWindowAndTrigger.stringKey()); }