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())); } }
/** * Advance the watermark of this source to the specified instant. * * <p>The watermark must advance monotonically and cannot advance to {@link * BoundedWindow#TIMESTAMP_MAX_VALUE} or beyond. * * @return A {@link TestStream.Builder} like this one that will advance the watermark to the * specified point after all earlier events have completed. */ public Builder<T> advanceWatermarkTo(Instant newWatermark) { checkArgument( !newWatermark.isBefore(currentWatermark), "The watermark must monotonically advance"); checkArgument( newWatermark.isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE), "The Watermark cannot progress beyond the maximum. Got: %s. Maximum: %s", newWatermark, BoundedWindow.TIMESTAMP_MAX_VALUE); ImmutableList<Event<T>> newEvents = ImmutableList.<Event<T>>builder() .addAll(events) .add(WatermarkEvent.advanceTo(newWatermark)) .build(); return new Builder<>(coder, newEvents, newWatermark); }
/** * Advance the watermark to infinity, completing this {@link TestStream}. Future calls to the * same builder will not affect the returned {@link TestStream}. */ public TestStream<T> advanceWatermarkToInfinity() { ImmutableList<Event<T>> newEvents = ImmutableList.<Event<T>>builder() .addAll(events) .add(WatermarkEvent.advanceTo(BoundedWindow.TIMESTAMP_MAX_VALUE)) .build(); return new TestStream<>(coder, newEvents); } }