You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/04/24 20:18:26 UTC

[GitHub] [beam] TheNeuralBit commented on a change in pull request #11477: [BEAM-9650] Add PeriodicSequence generator.

TheNeuralBit commented on a change in pull request #11477:
URL: https://github.com/apache/beam/pull/11477#discussion_r414822214



##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.io.range.OffsetRange;
+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.
+ *
+ * <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>All elements that have timestamp in the past will be output right away. Elements that have
+ * timestamp in the future will be delayed.

Review comment:
       This should be clear that past and future are determined by the system clock on the worker machine (can we just call that "processing time"?)

##########
File path: sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PeriodicImpulseTest.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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 PeriodicImpulse. */
+@RunWith(JUnit4.class)
+public class PeriodicImpulseTest {
+  @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(PeriodicImpulse.create().startAt(startTime).stopAt(stopTime).withInterval(interval))
+            .apply(ParDo.of(new ExtractTsDoFn<>()));
+
+    ArrayList<KV<Instant, Instant>> expectedResults =
+        new ArrayList<>((int) (duration / intervalMillis + 1));
+    for (long i = 0; i <= duration; i += intervalMillis) {
+      Instant el = startTime.plus(i);
+      expectedResults.add(KV.of(el, el));
+    }
+
+    PAssert.that(result).containsInAnyOrder(expectedResults);
+
+    p.run().waitUntilFinish();
+  }

Review comment:
       It would be good to add a test to verify `PeriodicImpulse` delays elements with timestamps in the future. One way could be to just set a startTime far in the future and verify nothing is output after some delay. But then the delay would have to be long enough to make sure the pipeline actually started for every runner, which isn't ideal.

##########
File path: sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PeriodicImpulseTest.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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 PeriodicImpulse. */
+@RunWith(JUnit4.class)
+public class PeriodicImpulseTest {
+  @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();

Review comment:
       It would be a lot easier to verify many different situations if there were some way to mock the clock, but that's pretty challenging when the clock might be on a remote worker. Have we solved that problem anywhere else?

##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.io.range.OffsetRange;
+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.
+ *
+ * <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

Review comment:
       Does the element type need to be a list? What about defining a type for it like:
   ```
   class SequenceDefinition {
     Instant first;
     Instant last;
     Duration interval;
     ...
   }
   ```

##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.io.range.OffsetRange;
+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.
+ *
+ * <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>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.

Review comment:
       I'm not sure I understand what this means, could you clarify? It looks like maybe it's re-stating the previous paragraph in a different way?

##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.io.range.OffsetRange;
+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.
+ *
+ * <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>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.
+ */
+@Experimental(Experimental.Kind.SPLITTABLE_DO_FN)
+public class PeriodicSequence extends PTransform<PCollection<List<Long>>, PCollection<Instant>> {

Review comment:
       Does this need to be public? If so, should it have a test as well?




----------------------------------------------------------------
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:
users@infra.apache.org