reuvenlax commented on a change in pull request #15786: URL: https://github.com/apache/beam/pull/15786#discussion_r771791868
########## File path: sdks/java/extensions/timeseries/src/test/java/org/apache/beam/sdk/extensions/timeseries/FillGapsTest.java ########## @@ -0,0 +1,357 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.timeseries; + +import com.google.auto.value.AutoValue; +import java.util.List; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Reify; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class FillGapsTest { + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + abstract static class Message { + abstract String getKey(); + + abstract String getValue(); + + abstract Instant getTimestamp(); + + static Message update(FillGaps.PropagateData<Message> propagateData) { + Message value = propagateData.getValue().getValue(); + Instant nextWindowMax = propagateData.getNextWindow().maxTimestamp(); + return value.toBuilder().setTimestamp(nextWindowMax).build(); + } + + static Message of(String key, String value, Instant timestamp) { + return new AutoValue_FillGapsTest_Message.Builder() + .setKey(key) + .setValue(value) + .setTimestamp(timestamp) + .build(); + } + + static TimestampedValue<Message> ofTimestamped(String key, String value, Instant timestamp) { + return TimestampedValue.of(of(key, value, timestamp), timestamp); + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setKey(String key); + + abstract Builder setValue(String value); + + abstract Builder setTimestamp(Instant timestamp); + + abstract Message build(); + } + + abstract Builder toBuilder(); + } + + private static class PrintingDoFn<T> extends DoFn<T, Void> { + @ProcessElement + public void processElement( + @Element T element, @Timestamp Instant timestamp, BoundedWindow window) { + System.err.println(element); + } + } + + @Test + public void testFillGaps() { + List<TimestampedValue<Message>> values = + ImmutableList.of( + Message.ofTimestamped("key1", "value0<", Instant.ofEpochSecond(0)), + Message.ofTimestamped("key2", "value0", Instant.ofEpochSecond(0)), + Message.ofTimestamped("key1", "value1<", Instant.ofEpochSecond(1)), + Message.ofTimestamped( + "key1", "value1", Instant.ofEpochSecond(1).plus(Duration.millis(1))), + Message.ofTimestamped("key2", "value1<", Instant.ofEpochSecond(1)), + Message.ofTimestamped( + "key2", "value1", Instant.ofEpochSecond(1).plus(Duration.millis(1))), + Message.ofTimestamped("key1", "value3", Instant.ofEpochSecond(3)), + Message.ofTimestamped("key2", "value3", Instant.ofEpochSecond(3))); + + PCollection<Message> input = pipeline.apply(Create.timestamped(values)); + PCollection<TimestampedValue<Message>> gapFilled = + input + .apply( + FillGaps.<Message>of(Duration.standardSeconds(1), "key") + .withStopTime(Instant.ofEpochSecond(5))) + .apply(Reify.timestamps()); + + FixedWindows fixedWindows = FixedWindows.of(Duration.standardSeconds(1)); + PAssert.that(gapFilled) + .containsInAnyOrder( + Iterables.concat( + values, + ImmutableList.of( + TimestampedValue.of( + Message.of( + "key1", "value1", Instant.ofEpochSecond(1).plus(Duration.millis(1))), + fixedWindows.assignWindow(Instant.ofEpochSecond(2)).maxTimestamp()), + TimestampedValue.of( + Message.of( + "key2", "value1", Instant.ofEpochSecond(1).plus(Duration.millis(1))), + fixedWindows.assignWindow(Instant.ofEpochSecond(2)).maxTimestamp()), + TimestampedValue.of( + Message.of("key1", "value3", Instant.ofEpochSecond(3)), + fixedWindows.assignWindow(Instant.ofEpochSecond(4)).maxTimestamp()), + TimestampedValue.of( + Message.of("key2", "value3", Instant.ofEpochSecond(3)), + fixedWindows.assignWindow(Instant.ofEpochSecond(4)).maxTimestamp())))); + pipeline.run(); + } + + @Test + public void testFillGapsKeepEarliest() { + List<TimestampedValue<Message>> values = + ImmutableList.of( + Message.ofTimestamped("key1", "value0<", Instant.ofEpochSecond(0)), + Message.ofTimestamped("key2", "value0", Instant.ofEpochSecond(0)), + Message.ofTimestamped("key1", "value1<", Instant.ofEpochSecond(1)), + Message.ofTimestamped( + "key1", "value1", Instant.ofEpochSecond(1).plus(Duration.millis(1))), + Message.ofTimestamped("key2", "value1<", Instant.ofEpochSecond(1)), + Message.ofTimestamped( + "key2", "value1", Instant.ofEpochSecond(1).plus(Duration.millis(1))), + Message.ofTimestamped("key1", "value3", Instant.ofEpochSecond(3)), + Message.ofTimestamped("key2", "value3", Instant.ofEpochSecond(3))); + + PCollection<Message> input = pipeline.apply(Create.timestamped(values)); + PCollection<TimestampedValue<Message>> gapFilled = + input + .apply( + FillGaps.<Message>of(Duration.standardSeconds(1), "key") + .withMergeFunction(FillGaps.keepEarliest()) + .withStopTime(Instant.ofEpochSecond(5))) + .apply(Reify.timestamps()); + + FixedWindows fixedWindows = FixedWindows.of(Duration.standardSeconds(1)); + PAssert.that(gapFilled) + .containsInAnyOrder( + Iterables.concat( + values, + ImmutableList.of( + TimestampedValue.of( + Message.of("key1", "value1<", Instant.ofEpochSecond(1)), + fixedWindows.assignWindow(Instant.ofEpochSecond(2)).maxTimestamp()), + TimestampedValue.of( + Message.of("key2", "value1<", Instant.ofEpochSecond(1)), + fixedWindows.assignWindow(Instant.ofEpochSecond(2)).maxTimestamp()), + TimestampedValue.of( + Message.of("key1", "value3", Instant.ofEpochSecond(3)), + fixedWindows.assignWindow(Instant.ofEpochSecond(4)).maxTimestamp()), + TimestampedValue.of( + Message.of("key2", "value3", Instant.ofEpochSecond(3)), + fixedWindows.assignWindow(Instant.ofEpochSecond(4)).maxTimestamp())))); + pipeline.run(); + } + + @Test + public void testFillGapsMaxDuration() { + List<TimestampedValue<Message>> values = + ImmutableList.of( + Message.ofTimestamped("key1", "value0", Instant.ofEpochSecond(0)), + Message.ofTimestamped("key2", "value0", Instant.ofEpochSecond(0)), + Message.ofTimestamped("key1", "value1", Instant.ofEpochSecond(1)), + Message.ofTimestamped("key2", "value1", Instant.ofEpochSecond(1)), + Message.ofTimestamped("key1", "value3", Instant.ofEpochSecond(10)), + Message.ofTimestamped("key2", "value3", Instant.ofEpochSecond(10))); + + PCollection<Message> input = pipeline.apply(Create.timestamped(values)); + PCollection<TimestampedValue<Message>> gapFilled = + input + .apply( + FillGaps.<Message>of(Duration.standardSeconds(1), "key") + .withMaxGapFillBuckets(4L) + .withStopTime(Instant.ofEpochSecond(11))) + .apply(Reify.timestamps()); + + FixedWindows fixedWindows = FixedWindows.of(Duration.standardSeconds(1)); + PAssert.that(gapFilled) + .containsInAnyOrder( + Iterables.concat( + values, + ImmutableList.of( + TimestampedValue.of( + Message.of("key1", "value1", Instant.ofEpochSecond(1)), + fixedWindows.assignWindow(Instant.ofEpochSecond(2)).maxTimestamp()), + TimestampedValue.of( + Message.of("key1", "value1", Instant.ofEpochSecond(1)), + fixedWindows.assignWindow(Instant.ofEpochSecond(3)).maxTimestamp()), + TimestampedValue.of( + Message.of("key1", "value1", Instant.ofEpochSecond(1)), + fixedWindows.assignWindow(Instant.ofEpochSecond(4)).maxTimestamp()), + TimestampedValue.of( + Message.of("key1", "value1", Instant.ofEpochSecond(1)), + fixedWindows.assignWindow(Instant.ofEpochSecond(5)).maxTimestamp()), + TimestampedValue.of( + Message.of("key2", "value1", Instant.ofEpochSecond(1)), + fixedWindows.assignWindow(Instant.ofEpochSecond(2)).maxTimestamp()), + TimestampedValue.of( + Message.of("key2", "value1", Instant.ofEpochSecond(1)), + fixedWindows.assignWindow(Instant.ofEpochSecond(3)).maxTimestamp()), + TimestampedValue.of( + Message.of("key2", "value1", Instant.ofEpochSecond(1)), + fixedWindows.assignWindow(Instant.ofEpochSecond(4)).maxTimestamp()), + TimestampedValue.of( + Message.of("key2", "value1", Instant.ofEpochSecond(1)), + fixedWindows.assignWindow(Instant.ofEpochSecond(5)).maxTimestamp())))); + pipeline.run(); + } + + @Test + public void testFillGapsPropagateFunction() { + List<TimestampedValue<Message>> values = + ImmutableList.of( + Message.ofTimestamped("key1", "value0<", Instant.ofEpochSecond(0)), + Message.ofTimestamped("key2", "value0", Instant.ofEpochSecond(0)), + Message.ofTimestamped("key1", "value1<", Instant.ofEpochSecond(1)), + Message.ofTimestamped( + "key1", "value1", Instant.ofEpochSecond(1).plus(Duration.millis(1))), + Message.ofTimestamped("key2", "value1<", Instant.ofEpochSecond(1)), + Message.ofTimestamped( + "key2", "value1", Instant.ofEpochSecond(1).plus(Duration.millis(1))), + Message.ofTimestamped("key1", "value3", Instant.ofEpochSecond(3)), + Message.ofTimestamped("key2", "value3", Instant.ofEpochSecond(3))); + + PCollection<Message> input = pipeline.apply(Create.timestamped(values)); + PCollection<TimestampedValue<Message>> gapFilled = + input + .apply( + FillGaps.<Message>of(Duration.standardSeconds(1), "key") + .withPropagateFunction(Message::update) + .withStopTime(Instant.ofEpochSecond(5))) + .apply(Reify.timestamps()); + + FixedWindows fixedWindows = FixedWindows.of(Duration.standardSeconds(1)); + Instant bucketTwoMax = fixedWindows.assignWindow(Instant.ofEpochSecond(2)).maxTimestamp(); + Instant bucketFourMax = fixedWindows.assignWindow(Instant.ofEpochSecond(4)).maxTimestamp(); + + PAssert.that(gapFilled) + .containsInAnyOrder( + Iterables.concat( + values, + ImmutableList.of( + Message.ofTimestamped("key1", "value1", bucketTwoMax), + Message.ofTimestamped("key2", "value1", bucketTwoMax), + Message.ofTimestamped("key1", "value3", bucketFourMax), + Message.ofTimestamped("key2", "value3", bucketFourMax)))); + pipeline.run(); + } + + // TODO: This test fails due to DirectRunner bugs. Uncomment once those bugs are fixed. Review comment: I've run it manually. Unfortunately if there were real bugs they would cause flakes, so simply retrying the test until it succeeds isn't helpful. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
