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 2021/08/30 21:51:02 UTC

[GitHub] [beam] kennknowles commented on a change in pull request #15378: [RFC] Define and document per-key ordering semantics for runners

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



##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesPerKeyOrderInStage.java
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.testing;
+
+import org.apache.beam.sdk.annotations.Internal;
+
+/**
+ * Category tag for validation tests which rely on a runner providing per-key ordering in between
+ * transforms in the same stage. Tests tagged with {@link UsesPerKeyOrderInStage} should be run for

Review comment:
       "Stage" is not a concept in the Beam model, unless you also define it elsewhere in this PR. (commenting as I go)

##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesPerKeyOrderedDelivery.java
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.testing;
+
+import org.apache.beam.sdk.annotations.Internal;
+
+/**
+ * Category tag for validation tests which rely on a runner providing per-key ordering. Tests tagged
+ * with {@link UsesPerKeyOrderedDelivery} should be run for runners which support key-to-key
+ * ordering of elements across shuffle / stage boundaries.

Review comment:
       "shuffle" is also not a model concept. For example runners can choose to move shuffles around so then the ordered delivery might change. Unless you can demonstrate/require that allowable optimizations preserve the ordered delivery you are looking for.

##########
File path: sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
##########
@@ -626,7 +627,7 @@ public void process(OutputReceiver<String> r, PipelineOptions options) {
     }
 
     @Test
-    @Category(ValidatesRunner.class)
+    @Category({ValidatesRunner.class, UsesParDoLifecycle.class})

Review comment:
       Seems unrelated.

##########
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:
       Java `assert` is, I think, less preferable than a library that generates more elaborate error messages. Also, failures within a user `DoFn` may be swallowed by a runner. Can the tests be described in terms of a dead-letter output and `PAssert`?




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org