static <T> TestStream.Event<T> eventFromProto( RunnerApi.TestStreamPayload.Event protoEvent, Coder<T> coder) throws IOException { switch (protoEvent.getEventCase()) { case WATERMARK_EVENT: return TestStream.WatermarkEvent.advanceTo( new Instant(protoEvent.getWatermarkEvent().getNewWatermark())); case PROCESSING_TIME_EVENT: return TestStream.ProcessingTimeEvent.advanceBy( Duration.millis(protoEvent.getProcessingTimeEvent().getAdvanceDuration())); case ELEMENT_EVENT: List<TimestampedValue<T>> decodedElements = new ArrayList<>(); for (RunnerApi.TestStreamPayload.TimestampedElement element : protoEvent.getElementEvent().getElementsList()) { decodedElements.add( TimestampedValue.of( CoderUtils.decodeFromByteArray(coder, element.getEncodedElement().toByteArray()), new Instant(element.getTimestamp()))); } return TestStream.ElementEvent.add(decodedElements); case EVENT_NOT_SET: default: throw new IllegalArgumentException( String.format( "Unsupported type of %s: %s", RunnerApi.TestStreamPayload.Event.class.getCanonicalName(), protoEvent.getEventCase())); } }
RunnerApi.TestStreamPayload.Event.AddElements.Builder builder = RunnerApi.TestStreamPayload.Event.AddElements.newBuilder(); for (TimestampedValue<T> element : ((TestStream.ElementEvent<T>) event).getElements()) { builder.addElements( RunnerApi.TestStreamPayload.TimestampedElement.newBuilder()
/** * Adds the specified elements to the source with the provided timestamps. * * @return A {@link TestStream.Builder} like this one that will add the provided elements after * all earlier events have completed. */ @SafeVarargs public final Builder<T> addElements( TimestampedValue<T> element, TimestampedValue<T>... elements) { checkArgument( element.getTimestamp().isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE), "Elements must have timestamps before %s. Got: %s", BoundedWindow.TIMESTAMP_MAX_VALUE, element.getTimestamp()); for (TimestampedValue<T> multiElement : elements) { checkArgument( multiElement.getTimestamp().isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE), "Elements must have timestamps before %s. Got: %s", BoundedWindow.TIMESTAMP_MAX_VALUE, multiElement.getTimestamp()); } ImmutableList<Event<T>> newEvents = ImmutableList.<Event<T>>builder() .addAll(events) .add(ElementEvent.add(element, elements)) .build(); return new Builder<>(coder, newEvents, currentWatermark); }
@Override public void processElement(WindowedValue<TestStreamIndex<T>> element) throws Exception { TestStreamIndex<T> streamIndex = element.getValue(); List<Event<T>> events = streamIndex.getTestStream().getEvents(); int index = streamIndex.getIndex(); Instant watermark = element.getTimestamp(); Event<T> event = events.get(index); if (event.getType().equals(EventType.ELEMENT)) { UncommittedBundle<T> bundle = context.createBundle( (PCollection<T>) Iterables.getOnlyElement(application.getOutputs().values())); for (TimestampedValue<T> elem : ((ElementEvent<T>) event).getElements()) { bundle.add( WindowedValue.timestampedValueInGlobalWindow(elem.getValue(), elem.getTimestamp())); } resultBuilder.addOutput(bundle); } if (event.getType().equals(EventType.WATERMARK)) { watermark = ((WatermarkEvent<T>) event).getWatermark(); } if (event.getType().equals(EventType.PROCESSING_TIME)) { ((TestClock) context.getClock()) .advance(((ProcessingTimeEvent<T>) event).getProcessingTimeAdvance()); } TestStreamIndex<T> next = streamIndex.next(); if (next.getIndex() < events.size()) { resultBuilder.addUnprocessedElements( Collections.singleton(WindowedValue.timestampedValueInGlobalWindow(next, watermark))); } }
@SafeVarargs static <T> Event<T> add(TimestampedValue<T> element, TimestampedValue<T>... elements) { return add(ImmutableList.<TimestampedValue<T>>builder().add(element).add(elements).build()); }