/** * Advance the processing time by the specified amount. * * @return A {@link TestStream.Builder} like this one that will advance the processing time by * the specified amount after all earlier events have completed. */ public Builder<T> advanceProcessingTime(Duration amount) { checkArgument( amount.getMillis() > 0, "Must advance the processing time by a positive amount. Got: ", amount); ImmutableList<Event<T>> newEvents = ImmutableList.<Event<T>>builder() .addAll(events) .add(ProcessingTimeEvent.advanceBy(amount)) .build(); return new Builder<>(coder, newEvents, currentWatermark); }
/** * 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); }
/** * 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); }
public static <T> Builder<T> create( Schema schema, SerializableFunction<T, Row> toRowFunction, SerializableFunction<Row, T> fromRowFunction) { return create(SchemaCoder.of(schema, toRowFunction, fromRowFunction)); }