@Override public void onMerge(OnMergeContext c) throws Exception { StateMerging.mergeBags(c.state(), bufferTag); } };
@Override public void onMerge(MergingStateAccessor<K, W> context) { StateMerging.mergeCombiningValues(context, PANE_ADDITIONS_TAG); } }
@Override public void prefetchOnMerge(MergingStateAccessor<K, W> state) throws Exception { StateMerging.prefetchBags(state, bufferTag); }
@Override public void onMerge(OnMergeContext context) throws Exception { // If we've already received enough elements and finished in some window, // then this trigger is just finished. if (context.trigger().finishedInAnyMergingWindow()) { context.trigger().setFinished(true); StateMerging.clear(context.state(), ELEMENTS_IN_PANE_TAG); return; } // Otherwise, compute the sum of elements in all the active panes. StateMerging.mergeCombiningValues(context.state(), ELEMENTS_IN_PANE_TAG); }
/** Merge all set state in {@code address} across all windows under merge. */ public static <K, T, W extends BoundedWindow> void mergeSets( MergingStateAccessor<K, W> context, StateTag<SetState<T>> address) { mergeSets(context.accessInEachMergingWindow(address).values(), context.access(address)); }
/** * Prefetch all combining value state for {@code address} across all merging windows in {@code * context}. */ public static <K, StateT extends GroupingState<?, ?>, W extends BoundedWindow> void prefetchCombiningValues(MergingStateAccessor<K, W> context, StateTag<StateT> address) { for (StateT state : context.accessInEachMergingWindow(address).values()) { prefetchRead(state); } }
@Override public void prefetchOnMerge(MergingStateAccessor<K, W> state) throws Exception { StateMerging.prefetchCombiningValues(state, bufferTag); }
StateMerging.clear(context.state(), EXTRA_HOLD_TAG); addGarbageCollectionHold(context, false /*paneIsEmpty*/);
@Override public void onMerge(OnMergeContext c) throws Exception { // NOTE: We could try to delete all timers which are still active, but we would // need access to a timer context for each merging window. // for (CombiningValueStateInternal<Instant, Combine.Holder<Instant>, Instant> state : // c.state().accessInEachMergingWindow(DELAYED_UNTIL_TAG).values()) { // Instant timestamp = state.get().read(); // if (timestamp != null) { // <context for merging window>.deleteTimer(timestamp, timeDomain); // } // } // Instead let them fire and be ignored. // If the trigger is already finished, there is no way it will become re-activated if (c.trigger().isFinished()) { StateMerging.clear(c.state(), DELAYED_UNTIL_TAG); // NOTE: We do not attempt to delete the timers. return; } // Determine the earliest point across all the windows, and delay to that. StateMerging.mergeCombiningValues(c.state(), DELAYED_UNTIL_TAG); Instant earliestTargetTime = c.state().access(DELAYED_UNTIL_TAG).read(); if (earliestTargetTime != null) { c.setTimer(earliestTargetTime, timeDomain); } }
@Test public void testMergeSetIntoSource() throws Exception { SetState<String> set1 = underTest.state(NAMESPACE_1, STRING_SET_ADDR); SetState<String> set2 = underTest.state(NAMESPACE_2, STRING_SET_ADDR); set1.add("Hello"); set2.add("Hello"); set2.add("World"); set1.add("!"); StateMerging.mergeSets(Arrays.asList(set1, set2), set1); // Reading the merged set gets both the contents assertThat(set1.read(), containsInAnyOrder("Hello", "World", "!")); assertThat(set2.read(), Matchers.emptyIterable()); }
/** * Prefetch all bag state in {@code address} across all windows under merge in {@code context}, * except for the bag state in the final state address window which we can blindly append to. */ public static <K, T, W extends BoundedWindow> void prefetchBags( MergingStateAccessor<K, W> context, StateTag<BagState<T>> address) { Map<W, BagState<T>> map = context.accessInEachMergingWindow(address); if (map.isEmpty()) { // Nothing to prefetch. return; } BagState<T> result = context.access(address); // Prefetch everything except what's already in result. for (BagState<T> source : map.values()) { if (!source.equals(result)) { prefetchRead(source); } } }
@Override public void prefetchOnMerge(MergingStateAccessor<K, W> state) { StateMerging.prefetchCombiningValues(state, PANE_ADDITIONS_TAG); }
@Override public void onMerge(OnMergeContext c) throws Exception { StateMerging.mergeCombiningValues(c.state(), bufferTag); } };
/** Merge all bag state in {@code address} across all windows under merge. */ public static <K, T, W extends BoundedWindow> void mergeBags( MergingStateAccessor<K, W> context, StateTag<BagState<T>> address) { mergeBags(context.accessInEachMergingWindow(address).values(), context.access(address)); }
@Test public void testMergeSetIntoNewNamespace() throws Exception { SetState<String> set1 = underTest.state(NAMESPACE_1, STRING_SET_ADDR); SetState<String> set2 = underTest.state(NAMESPACE_2, STRING_SET_ADDR); SetState<String> set3 = underTest.state(NAMESPACE_3, STRING_SET_ADDR); set1.add("Hello"); set2.add("Hello"); set2.add("World"); set1.add("!"); StateMerging.mergeSets(Arrays.asList(set1, set2, set3), set3); // Reading the merged set gets both the contents assertThat(set3.read(), containsInAnyOrder("Hello", "World", "!")); assertThat(set1.read(), Matchers.emptyIterable()); assertThat(set2.read(), Matchers.emptyIterable()); }
for (BagState<T> source : sources) { if (!source.equals(result)) { prefetchRead(source); futures.add(source);
@Override public void prefetchOnMerge(MergingStateAccessor<?, ?> state) { super.prefetchOnMerge(state); StateMerging.prefetchCombiningValues(state, DELAYED_UNTIL_TAG); }
/** Merge all value state in {@code address} across all merging windows in {@code context}. */ public static <K, InputT, AccumT, OutputT, W extends BoundedWindow> void mergeCombiningValues( MergingStateAccessor<K, W> context, StateTag<CombiningState<InputT, AccumT, OutputT>> address) { mergeCombiningValues( context.accessInEachMergingWindow(address).values(), context.access(address)); }
@Test public void testMergeBagIntoSource() throws Exception { BagState<String> bag1 = underTest.state(NAMESPACE_1, STRING_BAG_ADDR); BagState<String> bag2 = underTest.state(NAMESPACE_2, STRING_BAG_ADDR); bag1.add("Hello"); bag2.add("World"); bag1.add("!"); StateMerging.mergeBags(Arrays.asList(bag1, bag2), bag1); // Reading the merged bag gets both the contents assertThat(bag1.read(), containsInAnyOrder("Hello", "World", "!")); assertThat(bag2.read(), Matchers.emptyIterable()); }
for (SetState<T> source : sources) { if (!source.equals(result)) { prefetchRead(source); futures.add(source);