@Test public void testMergeCombiningValueIntoSource() throws Exception { CombiningState<Integer, int[], Integer> value1 = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); CombiningState<Integer, int[], Integer> value2 = underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR); value1.add(5); value2.add(10); value1.add(6); assertThat(value1.read(), equalTo(11)); assertThat(value2.read(), equalTo(10)); // Merging clears the old values and updates the result value. StateMerging.mergeCombiningValues(Arrays.asList(value1, value2), value1); assertThat(value1.read(), equalTo(21)); assertThat(value2.read(), equalTo(0)); }
accumulators.add(source.getAccum()); AccumT merged = result.mergeAccumulators(accumulators); source.clear(); result.addAccum(merged);
@Override public ReadableState<Boolean> isEmpty(StateAccessor<K> state) { return state.access(PANE_ADDITIONS_TAG).isEmpty(); }
@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; }
@Override public void recordContent(StateAccessor<K> state) { state.access(PANE_ADDITIONS_TAG).add(1L); }
@Override public void clearPane(StateAccessor<K> state) { state.access(PANE_ADDITIONS_TAG).clear(); }
@Override public boolean shouldFire(TriggerStateMachine.TriggerContext context) throws Exception { long count = context.state().access(ELEMENTS_IN_PANE_TAG).read(); return count >= countElems; }
@Override public void onElement(OnElementContext c) throws Exception { c.state().access(ELEMENTS_IN_PANE_TAG).add(1L); }
@Override public void clear(TriggerContext c) throws Exception { c.state().access(DELAYED_UNTIL_TAG).clear(); }
@ProcessElement public void processElement( @Element KV<String, Integer> element, @StateId(stateId) CombiningState<Integer, MyInteger, Integer> state, OutputReceiver<String> r) { state.add(element.getValue()); Integer currentValue = state.read(); if (currentValue == EXPECTED_SUM) { r.output("right on"); } } };
@Override public boolean shouldFire(TriggerStateMachine.TriggerContext context) throws Exception { Instant delayedUntil = context.state().access(DELAYED_UNTIL_TAG).read(); return delayedUntil != null && getCurrentTime(context) != null && getCurrentTime(context).isAfter(delayedUntil); }
/** * 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 void clear(TriggerContext c) throws Exception { c.state().access(ELEMENTS_IN_PANE_TAG).clear(); }
@ProcessElement public void processElement( @Element KV<String, Integer> element, @StateId(stateId) CombiningState<Integer, MyInteger, Integer> state, OutputReceiver<String> r) { state.add(element.getValue()); Integer currentValue = state.read(); if (currentValue == EXPECTED_SUM) { r.output("right on"); } } };
@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); } }
private void flushBatch( OutputReceiver<KV<K, Iterable<InputT>>> receiver, ValueState<K> key, BagState<InputT> batch, CombiningState<Long, long[], Long> numElementsInBatch) { Iterable<InputT> values = batch.read(); // when the timer fires, batch state might be empty if (!Iterables.isEmpty(values)) { receiver.output(KV.of(key.read(), values)); } batch.clear(); LOG.debug("*** BATCH *** clear"); numElementsInBatch.clear(); } }
@ProcessElement public void processElement( @TimerId(END_OF_WINDOW_ID) Timer timer, @StateId(BATCH_ID) BagState<InputT> batch, @StateId(NUM_ELEMENTS_IN_BATCH_ID) CombiningState<Long, long[], Long> numElementsInBatch, @StateId(KEY_ID) ValueState<K> key, @Element KV<K, InputT> element, BoundedWindow window, OutputReceiver<KV<K, Iterable<InputT>>> receiver) { Instant windowExpires = window.maxTimestamp().plus(allowedLateness); LOG.debug( "*** SET TIMER *** to point in time {} for window {}", windowExpires.toString(), window.toString()); timer.set(windowExpires); key.write(element.getKey()); batch.add(element.getValue()); LOG.debug("*** BATCH *** Add element for window {} ", window.toString()); // blind add is supported with combiningState numElementsInBatch.add(1L); Long num = numElementsInBatch.read(); if (num % prefetchFrequency == 0) { //prefetch data and modify batch state (readLater() modifies this) batch.readLater(); } if (num >= batchSize) { LOG.debug("*** END OF BATCH *** for window {}", window.toString()); flushBatch(receiver, key, batch, numElementsInBatch); } }
@Test public void testMergeCombiningValueIntoNewNamespace() throws Exception { CombiningState<Integer, int[], Integer> value1 = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); CombiningState<Integer, int[], Integer> value2 = underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR); CombiningState<Integer, int[], Integer> value3 = underTest.state(NAMESPACE_3, SUM_INTEGER_ADDR); value1.add(5); value2.add(10); value1.add(6); StateMerging.mergeCombiningValues(Arrays.asList(value1, value2), value3); // Merging clears the old values and updates the result value. assertThat(value1.read(), equalTo(0)); assertThat(value2.read(), equalTo(0)); assertThat(value3.read(), equalTo(21)); }
@ProcessElement public void processElement( ProcessContext c, @Element KV<String, Double> element, @StateId(stateId) CombiningState<Double, CountSum<Double>, Double> state, OutputReceiver<String> r) { state.add(element.getValue()); Double currentValue = state.read(); if (Math.abs(currentValue - 0.5) < EPSILON) { r.output("right on"); } } };
@ProcessElement public void processElement( @Element KV<String, Integer> element, @StateId(stateId) SetState<MyInteger> state, @StateId(countStateId) CombiningState<Integer, int[], Integer> count, OutputReceiver<Set<MyInteger>> r) { state.add(new MyInteger(element.getValue())); count.add(1); if (count.read() >= 4) { Set<MyInteger> set = Sets.newHashSet(state.read()); r.output(set); } } };