TheNeuralBit commented on a change in pull request #11477: URL: https://github.com/apache/beam/pull/11477#discussion_r417656677
########## File path: sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PeriodicSequenceTest.java ########## @@ -0,0 +1,82 @@ +/* + * 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.transforms; + +import java.util.ArrayList; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.UsesImpulse; +import org.apache.beam.sdk.testing.UsesStatefulParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for PeriodicSequence. */ +@RunWith(JUnit4.class) +public class PeriodicSequenceTest { + @Rule public transient TestPipeline p = TestPipeline.create(); + + public static class ExtractTsDoFn<InputT> extends DoFn<InputT, KV<InputT, Instant>> { + @ProcessElement + public void processElement(DoFn<InputT, KV<InputT, Instant>>.ProcessContext c) + throws Exception { + c.output(KV.of(c.element(), c.timestamp())); + } + } + + @Test + @Category({ + NeedsRunner.class, + UsesImpulse.class, + UsesStatefulParDo.class, + }) + public void testOutputsProperElements() { + Instant instant = Instant.now(); + + Instant startTime = instant.minus(Duration.standardHours(100)); + long duration = 500; + Duration interval = Duration.millis(250); + long intervalMillis = interval.getMillis(); + Instant stopTime = startTime.plus(duration); + + PCollection<KV<Instant, Instant>> result = + p.apply( + Create.<PeriodicSequence.SequenceDefinition>of( + new PeriodicSequence.SequenceDefinition(startTime, stopTime, interval))) + .apply(PeriodicSequence.create()) + .apply(ParDo.of(new ExtractTsDoFn<>())); Review comment: Can you add a comment here that the reason you put the Instant into a KV is so you can verify the timestamp? It took me a while to realize that. (Also if we don't already have some general purpose way to verify timestamps in PCollections maybe we should?) ########## File path: sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java ########## @@ -21,33 +21,69 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; -import java.util.List; +import java.util.Objects; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.sdk.schemas.JavaFieldSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; -import org.apache.beam.sdk.transforms.splittabledofn.Sizes; import org.apache.beam.sdk.transforms.splittabledofn.SplitResult; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; import org.joda.time.Duration; import org.joda.time.Instant; /** - * A {@link PTransform} which generates a sequence of timestamped elements at given interval in - * runtime. + * A {@link PTransform} which generates a sequence of timestamped elements at given runtime + * interval. * - * <p>Receives a PCollection<List<Long>> where each element triggers the generation of sequence and - * has following elements: 0: first element timestamp 1: last element timestamp 2: interval + * <p>Transform will not output elements prior to target time. Transform can output elements at any + * time after target time. * - * <p>All elements that have timestamp in the past will be output right away. Elements that have - * timestamp in the future will be delayed. - * - * <p>Transform will not output elements prior to target timestamp. Transform can output elements at - * any time after target timestamp. + * <p>Multiple elements can be output at given moment if their timestamp is earlier than current + * time. */ @Experimental(Experimental.Kind.SPLITTABLE_DO_FN) -public class PeriodicSequence extends PTransform<PCollection<List<Long>>, PCollection<Instant>> { +public class PeriodicSequence + extends PTransform<PCollection<PeriodicSequence.SequenceDefinition>, PCollection<Instant>> { + + @DefaultSchema(JavaFieldSchema.class) + public static class SequenceDefinition { + public Instant first; + public Instant last; + public Long durationMilliSec; + + public SequenceDefinition() {} + + public SequenceDefinition(Instant first, Instant last, Duration duration) { + this.first = first; + this.last = last; + this.durationMilliSec = duration.getMillis(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || obj.getClass() != this.getClass()) { + return false; + } + + SequenceDefinition src = (SequenceDefinition) obj; + return src.first.equals(this.first) + && src.last.equals(this.last) + && src.durationMilliSec.equals(this.durationMilliSec); + } + + @Override + public int hashCode() { + int result = Objects.hash(first, last, durationMilliSec); + return result; + } Review comment: nit: Consider using `AutoValue` (and `AutoValueSchemaProvider` if you use schemas) so you don't have to implement these. Up to you though. ########## File path: sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java ########## @@ -21,33 +21,69 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; -import java.util.List; +import java.util.Objects; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.sdk.schemas.JavaFieldSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; -import org.apache.beam.sdk.transforms.splittabledofn.Sizes; import org.apache.beam.sdk.transforms.splittabledofn.SplitResult; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; import org.joda.time.Duration; import org.joda.time.Instant; /** - * A {@link PTransform} which generates a sequence of timestamped elements at given interval in - * runtime. + * A {@link PTransform} which generates a sequence of timestamped elements at given runtime + * interval. * - * <p>Receives a PCollection<List<Long>> where each element triggers the generation of sequence and - * has following elements: 0: first element timestamp 1: last element timestamp 2: interval + * <p>Transform will not output elements prior to target time. Transform can output elements at any + * time after target time. * - * <p>All elements that have timestamp in the past will be output right away. Elements that have - * timestamp in the future will be delayed. - * - * <p>Transform will not output elements prior to target timestamp. Transform can output elements at - * any time after target timestamp. + * <p>Multiple elements can be output at given moment if their timestamp is earlier than current + * time. */ @Experimental(Experimental.Kind.SPLITTABLE_DO_FN) -public class PeriodicSequence extends PTransform<PCollection<List<Long>>, PCollection<Instant>> { +public class PeriodicSequence + extends PTransform<PCollection<PeriodicSequence.SequenceDefinition>, PCollection<Instant>> { + + @DefaultSchema(JavaFieldSchema.class) + public static class SequenceDefinition { + public Instant first; + public Instant last; + public Long durationMilliSec; Review comment: We really should have support for `Duration` in beam schemas... filed [BEAM-9859](https://issues.apache.org/jira/browse/BEAM-9859) for this. In the meantime, couldn't you just use `@DefaultCoder(SerializableCoder.class)` so you could store this as a Duration? ########## File path: sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java ########## @@ -0,0 +1,212 @@ +/* + * 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.transforms; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; + +import java.util.Objects; +import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.sdk.schemas.JavaFieldSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.transforms.splittabledofn.SplitResult; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** + * A {@link PTransform} which generates a sequence of timestamped elements at given runtime + * interval. + * + * <p>Transform will not output elements prior to target time. Transform can output elements at any + * time after target time. Review comment: Can you add a note that this is according to the clock on the worker node? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected]
