private InProcessEvaluationContext( InProcessPipelineOptions options, BundleFactory bundleFactory, Collection<AppliedPTransform<?, ?, ?>> rootTransforms, Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers, Map<AppliedPTransform<?, ?, ?>, String> stepNames, Collection<PCollectionView<?>> views) { this.options = checkNotNull(options); this.bundleFactory = checkNotNull(bundleFactory); checkNotNull(rootTransforms); checkNotNull(valueToConsumers); checkNotNull(stepNames); checkNotNull(views); this.stepNames = stepNames; this.watermarkManager = InMemoryWatermarkManager.create( NanosOffsetClock.create(), rootTransforms, valueToConsumers); this.sideInputContainer = InProcessSideInputContainer.create(this, views); this.applicationStateInternals = new ConcurrentHashMap<>(); this.mergedCounters = new CounterSet(); this.callbackExecutor = WatermarkCallbackExecutor.create(Executors.newSingleThreadExecutor()); }
/** Get an {@link ExecutionContext} for the provided {@link AppliedPTransform} and key. */ public InProcessExecutionContext getExecutionContext( AppliedPTransform<?, ?, ?> application, Object key) { StepAndKey stepAndKey = StepAndKey.of(application, key); return new InProcessExecutionContext( options.getClock(), key, (CopyOnAccessInMemoryStateInternals<Object>) applicationStateInternals.get(stepAndKey), watermarkManager.getWatermarks(application)); }
private Iterable<? extends CommittedBundle<?>> commitBundles( Iterable<? extends UncommittedBundle<?>> bundles) { ImmutableList.Builder<CommittedBundle<?>> completed = ImmutableList.builder(); for (UncommittedBundle<?> inProgress : bundles) { AppliedPTransform<?, ?, ?> producing = inProgress.getPCollection().getProducingTransformInternal(); TransformWatermarks watermarks = watermarkManager.getWatermarks(producing); CommittedBundle<?> committed = inProgress.commit(watermarks.getSynchronizedProcessingOutputTime()); // Empty bundles don't impact watermarks and shouldn't trigger downstream execution, so // filter them out if (!Iterables.isEmpty(committed.getElements())) { completed.add(committed); } } return completed.build(); }
/** * 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; }
/** * Returns true if the step will not produce additional output. * * <p>If the provided transform produces only {@link IsBounded#BOUNDED} {@link PCollection * PCollections}, returns true if the watermark is at {@link BoundedWindow#TIMESTAMP_MAX_VALUE * positive infinity}. * * <p>If the provided transform produces any {@link IsBounded#UNBOUNDED} {@link PCollection * PCollections}, returns the value of {@link * InProcessPipelineOptions#isShutdownUnboundedProducersWithMaxWatermark()}. */ public boolean isDone(AppliedPTransform<?, ?, ?> transform) { // if the PTransform's watermark isn't at the max value, it isn't done if (watermarkManager .getWatermarks(transform) .getOutputWatermark() .isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE)) { return false; } // If the PTransform has any unbounded outputs, and unbounded producers should not be shut down, // the PTransform may produce additional output. It is not done. for (PValue output : transform.getOutput().expand()) { if (output instanceof PCollection) { IsBounded bounded = ((PCollection<?>) output).isBounded(); if (bounded.equals(IsBounded.UNBOUNDED) && !options.isShutdownUnboundedProducersWithMaxWatermark()) { return false; } } } // The PTransform's watermark was at positive infinity and all of its outputs are known to be // done. It is done. return true; }
/** * Extracts all timers that have been fired and have not already been extracted. * * <p>This is a destructive operation. Timers will only appear in the result of this method once * for each time they are set. */ public Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> extractFiredTimers() { forceRefresh(); Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> fired = watermarkManager.extractFiredTimers(); return fired; }
@VisibleForTesting void forceRefresh() { watermarkManager.refreshAll(); fireAllAvailableCallbacks(); }
private void fireAvailableCallbacks(AppliedPTransform<?, ?, ?> producingTransform) { TransformWatermarks watermarks = watermarkManager.getWatermarks(producingTransform); callbackExecutor.fireForWatermark(producingTransform, watermarks.getOutputWatermark()); }