pabloem commented on a change in pull request #15378:
URL: https://github.com/apache/beam/pull/15378#discussion_r698703596



##########
File path: 
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PerKeyOrderingTest.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.UsesPerKeyOrderInStage;
+import org.apache.beam.sdk.testing.UsesPerKeyOrderedDelivery;
+import org.apache.beam.sdk.testing.ValidatesRunner;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@RunWith(JUnit4.class)
+public class PerKeyOrderingTest implements Serializable {
+  @Rule public final transient TestPipeline pipeline = TestPipeline.create();
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(PerKeyOrderingTest.class);
+
+  private static class VerifyDoFn<T> extends DoFn<KV<String, T>, KV<String, 
Boolean>> {
+    private final List<T> perKeyElements;
+
+    VerifyDoFn(List<T> perKeyElements) {
+      this.perKeyElements = perKeyElements;
+    }
+
+    @StateId("matchedElements")
+    private final StateSpec<ValueState<Integer>> elementsMatchedState = 
StateSpecs.value();
+
+    @ProcessElement
+    public void verifyOrder(
+        @Element KV<String, T> elm,
+        OutputReceiver<KV<String, Boolean>> receiver,
+        @StateId("matchedElements") ValueState<Integer> matchedElements) {
+      Integer matched = matchedElements.read();
+      matched = matched == null ? 0 : matched;
+      if (matched == -1) {
+        // When matched is set to -1, it means that we have met an error, and 
elements on this
+        // key are not matched anymore - thus we ignore all inputs.
+      } else if (matched < this.perKeyElements.size()
+          && !this.perKeyElements.get(matched).equals(elm.getValue())) {
+        // If we meet this condition, then the order of elements is not what 
we're expecting.
+        // We mark `matched` as -1, and output a failed ordering.
+        matchedElements.write(-1);
+        receiver.output(KV.of(elm.getKey(), false));
+      } else {
+        assert this.perKeyElements.get(matched).equals(elm.getValue())

Review comment:
       the case above is denied (`!...`?)

##########
File path: 
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PerKeyOrderingTest.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.UsesPerKeyOrderInStage;
+import org.apache.beam.sdk.testing.UsesPerKeyOrderedDelivery;
+import org.apache.beam.sdk.testing.ValidatesRunner;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@RunWith(JUnit4.class)
+public class PerKeyOrderingTest implements Serializable {
+  @Rule public final transient TestPipeline pipeline = TestPipeline.create();
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(PerKeyOrderingTest.class);
+
+  private static class VerifyDoFn<T> extends DoFn<KV<String, T>, KV<String, 
Boolean>> {
+    private final List<T> perKeyElements;
+
+    VerifyDoFn(List<T> perKeyElements) {
+      this.perKeyElements = perKeyElements;
+    }
+
+    @StateId("matchedElements")
+    private final StateSpec<ValueState<Integer>> elementsMatchedState = 
StateSpecs.value();
+
+    @ProcessElement
+    public void verifyOrder(
+        @Element KV<String, T> elm,
+        OutputReceiver<KV<String, Boolean>> receiver,
+        @StateId("matchedElements") ValueState<Integer> matchedElements) {
+      Integer matched = matchedElements.read();
+      matched = matched == null ? 0 : matched;
+      if (matched == -1) {
+        // When matched is set to -1, it means that we have met an error, and 
elements on this
+        // key are not matched anymore - thus we ignore all inputs.

Review comment:
       done

##########
File path: 
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PerKeyOrderingTest.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.UsesPerKeyOrderInStage;
+import org.apache.beam.sdk.testing.UsesPerKeyOrderedDelivery;
+import org.apache.beam.sdk.testing.ValidatesRunner;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@RunWith(JUnit4.class)
+public class PerKeyOrderingTest implements Serializable {
+  @Rule public final transient TestPipeline pipeline = TestPipeline.create();
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(PerKeyOrderingTest.class);
+
+  private static class VerifyDoFn<T> extends DoFn<KV<String, T>, KV<String, 
Boolean>> {
+    private final List<T> perKeyElements;
+
+    VerifyDoFn(List<T> perKeyElements) {
+      this.perKeyElements = perKeyElements;
+    }
+
+    @StateId("matchedElements")
+    private final StateSpec<ValueState<Integer>> elementsMatchedState = 
StateSpecs.value();
+
+    @ProcessElement
+    public void verifyOrder(
+        @Element KV<String, T> elm,
+        OutputReceiver<KV<String, Boolean>> receiver,
+        @StateId("matchedElements") ValueState<Integer> matchedElements) {
+      Integer matched = matchedElements.read();
+      matched = matched == null ? 0 : matched;
+      if (matched == -1) {
+        // When matched is set to -1, it means that we have met an error, and 
elements on this
+        // key are not matched anymore - thus we ignore all inputs.
+      } else if (matched < this.perKeyElements.size()

Review comment:
       done.

##########
File path: 
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PerKeyOrderingTest.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.UsesPerKeyOrderInStage;
+import org.apache.beam.sdk.testing.UsesPerKeyOrderedDelivery;
+import org.apache.beam.sdk.testing.ValidatesRunner;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@RunWith(JUnit4.class)
+public class PerKeyOrderingTest implements Serializable {
+  @Rule public final transient TestPipeline pipeline = TestPipeline.create();
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(PerKeyOrderingTest.class);
+
+  private static class VerifyDoFn<T> extends DoFn<KV<String, T>, KV<String, 
Boolean>> {
+    private final List<T> perKeyElements;
+
+    VerifyDoFn(List<T> perKeyElements) {
+      this.perKeyElements = perKeyElements;
+    }
+
+    @StateId("matchedElements")
+    private final StateSpec<ValueState<Integer>> elementsMatchedState = 
StateSpecs.value();
+
+    @ProcessElement
+    public void verifyOrder(
+        @Element KV<String, T> elm,
+        OutputReceiver<KV<String, Boolean>> receiver,
+        @StateId("matchedElements") ValueState<Integer> matchedElements) {
+      Integer matched = matchedElements.read();
+      matched = matched == null ? 0 : matched;
+      if (matched == -1) {
+        // When matched is set to -1, it means that we have met an error, and 
elements on this
+        // key are not matched anymore - thus we ignore all inputs.
+      } else if (matched < this.perKeyElements.size()
+          && !this.perKeyElements.get(matched).equals(elm.getValue())) {
+        // If we meet this condition, then the order of elements is not what 
we're expecting.
+        // We mark `matched` as -1, and output a failed ordering.
+        matchedElements.write(-1);
+        receiver.output(KV.of(elm.getKey(), false));
+      } else {
+        assert this.perKeyElements.get(matched).equals(elm.getValue())
+            : String.format("Element %s is not expected %s", elm, 
this.perKeyElements.get(matched));
+        matchedElements.write(matched + 1);
+        // If we reached the end of perKeyElements, it means that all elements 
have been emitted in
+        // the expected order, and thus we mark this Key as successful.
+        if (matched + 1 == perKeyElements.size()) {
+          receiver.output(KV.of(elm.getKey(), true));
+        }
+      }
+    }
+  }
+
+  @Test
+  @Category({ValidatesRunner.class, UsesPerKeyOrderedDelivery.class})
+  public void testSingleCallOrderingWithShuffle() {
+    // Here we test that the output of a single process call in a DoFn will be 
output in order
+    List<Integer> perKeyElements =
+        Lists.newArrayList(-8, -7, -6, -5, -4, -3, -2, -1, 0, 1, 2, 3, 4, 5, 
6, 7, 8);
+    Collections.shuffle(perKeyElements);
+    List<String> allKeys =
+        Lists.newArrayList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10).stream()
+            .map(elm -> String.format("k%s", elm))
+            .collect(Collectors.toList());
+    PCollection<KV<String, Integer>> kvSeeds =

Review comment:
       hmmm ... I could try to parameterize the test with different types 
-though IDK if the `Parameterized` JUnit runner integrates well with the 
ValidatesRunner framework. Do you have a suggestion? Should I try adding a more 
complex type instead of just ints?

##########
File path: 
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PerKeyOrderingTest.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.UsesPerKeyOrderInStage;
+import org.apache.beam.sdk.testing.UsesPerKeyOrderedDelivery;
+import org.apache.beam.sdk.testing.ValidatesRunner;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@RunWith(JUnit4.class)
+public class PerKeyOrderingTest implements Serializable {
+  @Rule public final transient TestPipeline pipeline = TestPipeline.create();
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(PerKeyOrderingTest.class);
+
+  private static class VerifyDoFn<T> extends DoFn<KV<String, T>, KV<String, 
Boolean>> {
+
+    private final List<T> perKeyElements;
+
+    VerifyDoFn(List<T> perKeyElements) {
+      this.perKeyElements = perKeyElements;
+    }
+
+    @StateId("matchedElements")
+    private final StateSpec<ValueState<Integer>> elementsMatchedState = 
StateSpecs.value();
+
+    @ProcessElement
+    public void verifyOrder(
+        @Element KV<String, T> elm,
+        OutputReceiver<KV<String, Boolean>> receiver,
+        @StateId("matchedElements") ValueState<Integer> matchedElements) {
+      Integer matched = matchedElements.read();
+      matched = matched == null ? 0 : matched;
+      if (matched == -1) {
+        // When matched is set to -1, it means that we have met an error, and 
elements on this
+        // key are not matched anymore - thus we ignore all inputs.
+        return;
+      } else if (matched < this.perKeyElements.size()
+          && !this.perKeyElements.get(matched).equals(elm.getValue())) {
+        // If we meet this condition, then the order of elements is not what 
we're expecting.
+        // We mark `matched` as -1, and output a failed ordering.
+        matchedElements.write(-1);
+        receiver.output(KV.of(elm.getKey(), false));
+      } else if (matched >= this.perKeyElements.size()) {
+        throw new RuntimeException("Got more elements than expected!");
+      } else {
+        assert this.perKeyElements.get(matched).equals(elm.getValue())
+            : String.format("Element %s is not expected %s", elm, 
this.perKeyElements.get(matched));
+        matchedElements.write(matched + 1);
+        // If we reached the end of perKeyElements, it means that all elements 
have been emitted in
+        // the expected order, and thus we mark this Key as successful.
+        if (matched + 1 == perKeyElements.size()) {
+          receiver.output(KV.of(elm.getKey(), true));
+        }
+      }
+    }
+  }
+
+  @Test
+  @Category({ValidatesRunner.class, UsesPerKeyOrderedDelivery.class})
+  public void testSingleCallOrderingWithShuffle() {
+    // Here we test that the output of a single process call in a DoFn will be 
output in order
+    List<Integer> perKeyElements =
+        Lists.newArrayList(-8, -7, -6, -5, -4, -3, -2, -1, 0, 1, 2, 3, 4, 5, 
6, 7, 8);

Review comment:
       I tried that. All good. 




-- 
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]


Reply via email to