/** * Handle the provided {@link InProcessTransformResult}, produced after evaluating the provided * {@link CommittedBundle} (potentially null, if the result of a root {@link PTransform}). * * <p>The result is the output of running the transform contained in the {@link * InProcessTransformResult} on the contents of the provided bundle. * * @param completedBundle the bundle that was processed to produce the result. Potentially {@code * null} if the transform that produced the result is a root transform * @param completedTimers the timers that were delivered to produce the {@code completedBundle}, * or an empty iterable if no timers were delivered * @param result the result of evaluating the input bundle * @return the committed bundles contained within the handled {@code result} */ public CommittedResult handleResult( @Nullable CommittedBundle<?> completedBundle, Iterable<TimerData> completedTimers, InProcessTransformResult result) { Iterable<? extends CommittedBundle<?>> committedBundles = commitBundles(result.getOutputBundles()); // Update watermarks and timers EnumSet<OutputType> outputTypes = EnumSet.copyOf(result.getOutputTypes()); if (Iterables.isEmpty(committedBundles)) { outputTypes.remove(OutputType.BUNDLE); } else { outputTypes.add(OutputType.BUNDLE); } CommittedResult committedResult = CommittedResult.create( result, completedBundle == null ? null : completedBundle.withElements((Iterable) result.getUnprocessedElements()), committedBundles, outputTypes); watermarkManager.updateWatermarks( completedBundle, result.getTimerUpdate().withCompletedTimers(completedTimers), committedResult, result.getWatermarkHold()); // Update counters if (result.getCounters() != null) { mergedCounters.merge(result.getCounters()); } // Update state internals CopyOnAccessInMemoryStateInternals<?> theirState = result.getState(); if (theirState != null) { CopyOnAccessInMemoryStateInternals<?> committedState = theirState.commit(); StepAndKey stepAndKey = StepAndKey.of( result.getTransform(), completedBundle == null ? null : completedBundle.getKey()); if (!committedState.isEmpty()) { applicationStateInternals.put(stepAndKey, committedState); } else { applicationStateInternals.remove(stepAndKey); } } return committedResult; }