public static Trigger fromProto(RunnerApi.Trigger triggerProto) { switch (triggerProto.getTriggerCase()) { case AFTER_ALL: return AfterAll.of(protosToTriggers(triggerProto.getAfterAll().getSubtriggersList())); case AFTER_ANY: return AfterFirst.of(protosToTriggers(triggerProto.getAfterAny().getSubtriggersList())); case AFTER_EACH: return AfterEach.inOrder( protosToTriggers(triggerProto.getAfterEach().getSubtriggersList())); case AFTER_END_OF_WINDOW: RunnerApi.Trigger.AfterEndOfWindow eowProto = triggerProto.getAfterEndOfWindow(); .withEarlyFirings( (OnceTrigger) fromProto(triggerProto.getAfterEndOfWindow().getEarlyFirings())); (OnceTrigger) fromProto(triggerProto.getAfterEndOfWindow().getLateFirings())); .withLateFirings((OnceTrigger) fromProto(eowProto.getLateFirings())); return Never.ever(); case OR_FINALLY: return fromProto(triggerProto.getOrFinally().getMain()) .orFinally((OnceTrigger) fromProto(triggerProto.getOrFinally().getFinally())); case REPEAT: return Repeatedly.forever(fromProto(triggerProto.getRepeat().getSubtrigger())); case DEFAULT: return DefaultTrigger.of();
private static List<Trigger> protosToTriggers(List<RunnerApi.Trigger> triggers) { List<Trigger> result = Lists.newArrayList(); for (RunnerApi.Trigger trigger : triggers) { result.add(fromProto(trigger)); } return result; }
/** * Converts from {@link RunnerApi.WindowingStrategy} to the SDK's {@link WindowingStrategy} using * the provided components to dereferences identifiers found in the proto. */ public static WindowingStrategy<?, ?> fromProto( RunnerApi.WindowingStrategy proto, RehydratedComponents components) throws InvalidProtocolBufferException { SdkFunctionSpec windowFnSpec = proto.getWindowFn(); WindowFn<?, ?> windowFn = windowFnFromProto(windowFnSpec); TimestampCombiner timestampCombiner = timestampCombinerFromProto(proto.getOutputTime()); AccumulationMode accumulationMode = fromProto(proto.getAccumulationMode()); Trigger trigger = TriggerTranslation.fromProto(proto.getTrigger()); ClosingBehavior closingBehavior = fromProto(proto.getClosingBehavior()); Duration allowedLateness = Duration.millis(proto.getAllowedLateness()); OnTimeBehavior onTimeBehavior = fromProto(proto.getOnTimeBehavior()); return WindowingStrategy.of(windowFn) .withAllowedLateness(allowedLateness) .withMode(accumulationMode) .withTrigger(trigger) .withTimestampCombiner(timestampCombiner) .withClosingBehavior(closingBehavior) .withOnTimeBehavior(onTimeBehavior); }
private RunnerApi.Trigger convertSpecific(AfterWatermarkEarlyAndLate v) { RunnerApi.Trigger.AfterEndOfWindow.Builder builder = RunnerApi.Trigger.AfterEndOfWindow.newBuilder(); builder.setEarlyFirings(toProto(v.getEarlyTrigger())); if (v.getLateTrigger() != null) { builder.setLateFirings(toProto(v.getLateTrigger())); } return RunnerApi.Trigger.newBuilder().setAfterEndOfWindow(builder).build(); }
private RunnerApi.Trigger convertSpecific(Repeatedly v) { return RunnerApi.Trigger.newBuilder() .setRepeat( RunnerApi.Trigger.Repeat.newBuilder().setSubtrigger(toProto(v.getRepeatedTrigger()))) .build(); }
private RunnerApi.Trigger convertSpecific(AfterEach v) { RunnerApi.Trigger.AfterEach.Builder builder = RunnerApi.Trigger.AfterEach.newBuilder(); for (Trigger subtrigger : v.subTriggers()) { builder.addSubtriggers(toProto(subtrigger)); } return RunnerApi.Trigger.newBuilder().setAfterEach(builder).build(); }
private RunnerApi.Trigger convertSpecific(AfterFirst v) { RunnerApi.Trigger.AfterAny.Builder builder = RunnerApi.Trigger.AfterAny.newBuilder(); for (Trigger subtrigger : v.subTriggers()) { builder.addSubtriggers(toProto(subtrigger)); } return RunnerApi.Trigger.newBuilder().setAfterAny(builder).build(); }
private RunnerApi.Trigger convertSpecific(AfterAll v) { RunnerApi.Trigger.AfterAll.Builder builder = RunnerApi.Trigger.AfterAll.newBuilder(); for (Trigger subtrigger : v.subTriggers()) { builder.addSubtriggers(toProto(subtrigger)); } return RunnerApi.Trigger.newBuilder().setAfterAll(builder).build(); }
ExecutableTriggerStateMachine.create( TriggerStateMachines.stateMachineForTrigger( TriggerTranslation.toProto(windowingStrategy.getTrigger()))), stateInternalsFactory.stateInternalsForKey(key), timerInternals,
@ProcessElement public void processElement(ProcessContext c) throws Exception { KeyedWorkItem<K, InputT> keyedWorkItem = c.element(); K key = keyedWorkItem.key(); StateInternals stateInternals = stateInternalsFactory.stateInternalsForKey(key); TimerInternals timerInternals = timerInternalsFactory.timerInternalsForKey(key); ReduceFnRunner<K, InputT, OutputT, W> reduceFnRunner = new ReduceFnRunner<>( key, windowingStrategy, ExecutableTriggerStateMachine.create( TriggerStateMachines.stateMachineForTrigger( TriggerTranslation.toProto(windowingStrategy.getTrigger()))), stateInternals, timerInternals, outputWindowedValue(), sideInputReader, reduceFn, c.getPipelineOptions()); reduceFnRunner.processElements(keyedWorkItem.elementsIterable()); reduceFnRunner.onTimers(keyedWorkItem.timersIterable()); reduceFnRunner.persist(); } }
public static <W extends BoundedWindow, AccumT, OutputT> ReduceFnTester<Integer, OutputT, W> combining( WindowingStrategy<?, W> strategy, CombineFnWithContext<Integer, AccumT, OutputT> combineFn, Coder<OutputT> outputCoder, PipelineOptions options, SideInputReader sideInputReader) throws Exception { CoderRegistry registry = CoderRegistry.createDefault(); // Ensure that the CombineFn can be converted into an AppliedCombineFn AppliedCombineFn.withInputCoder( combineFn, registry, KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())); return combining( strategy, TriggerStateMachines.stateMachineForTrigger( TriggerTranslation.toProto(strategy.getTrigger())), combineFn, outputCoder, options, sideInputReader); }
/** * Creates a {@link ReduceFnTester} for the given {@link WindowingStrategy} and {@link CombineFn}, * creating a {@link TriggerStateMachine} from the {@link Trigger} in the {@link * WindowingStrategy}. */ public static <W extends BoundedWindow, AccumT, OutputT> ReduceFnTester<Integer, OutputT, W> combining( WindowingStrategy<?, W> strategy, CombineFn<Integer, AccumT, OutputT> combineFn, Coder<OutputT> outputCoder) throws Exception { CoderRegistry registry = CoderRegistry.createDefault(); // Ensure that the CombineFn can be converted into an AppliedCombineFn AppliedCombineFn.withInputCoder( combineFn, registry, KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())); return combining( strategy, TriggerStateMachines.stateMachineForTrigger( TriggerTranslation.toProto(strategy.getTrigger())), combineFn, outputCoder); }
/** * Creates a {@link ReduceFnTester} for the given {@link WindowingStrategy}, creating a {@link * TriggerStateMachine} from its {@link Trigger}. */ public static <W extends BoundedWindow> ReduceFnTester<Integer, Iterable<Integer>, W> nonCombining( WindowingStrategy<?, W> windowingStrategy) throws Exception { return new ReduceFnTester<>( windowingStrategy, TriggerStateMachines.stateMachineForTrigger( TriggerTranslation.toProto(windowingStrategy.getTrigger())), SystemReduceFn.buffering(VarIntCoder.of()), IterableCoder.of(VarIntCoder.of()), PipelineOptionsFactory.create(), NullSideInputReader.empty()); }
/** * Converts a {@link WindowingStrategy} into a {@link RunnerApi.WindowingStrategy}, registering * any components in the provided {@link SdkComponents}. */ public static RunnerApi.WindowingStrategy toProto( WindowingStrategy<?, ?> windowingStrategy, SdkComponents components) throws IOException { SdkFunctionSpec windowFnSpec = toProto(windowingStrategy.getWindowFn(), components); RunnerApi.WindowingStrategy.Builder windowingStrategyProto = RunnerApi.WindowingStrategy.newBuilder() .setOutputTime(toProto(windowingStrategy.getTimestampCombiner())) .setAccumulationMode(toProto(windowingStrategy.getMode())) .setClosingBehavior(toProto(windowingStrategy.getClosingBehavior())) .setAllowedLateness(windowingStrategy.getAllowedLateness().getMillis()) .setTrigger(TriggerTranslation.toProto(windowingStrategy.getTrigger())) .setWindowFn(windowFnSpec) .setAssignsToOneWindow(windowingStrategy.getWindowFn().assignsToOneWindow()) .setOnTimeBehavior(toProto(windowingStrategy.getOnTimeBehavior())) .setWindowCoderId( components.registerCoder(windowingStrategy.getWindowFn().windowCoder())); return windowingStrategyProto.build(); }
ExecutableTriggerStateMachine.create( TriggerStateMachines.stateMachineForTrigger( TriggerTranslation.toProto(windowingStrategy.getTrigger()))), stateInternals, timerInternals,