@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); }
@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); } }
StateMerging.clear(context.state(), EXTRA_HOLD_TAG); addGarbageCollectionHold(context, false /*paneIsEmpty*/);