[
https://issues.apache.org/jira/browse/BEAM-3194?focusedWorklogId=138524&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-138524
]
ASF GitHub Bot logged work on BEAM-3194:
----------------------------------------
Author: ASF GitHub Bot
Created on: 27/Aug/18 19:39
Start Date: 27/Aug/18 19:39
Worklog Time Spent: 10m
Work Description: alanmyrvold commented on a change in pull request
#6220: [BEAM-3194] Add ValidatesRunner test for support of @RequiresStableInput
URL: https://github.com/apache/beam/pull/6220#discussion_r213090320
##########
File path:
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoRequiresStableInputTest.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.Charset;
+import java.util.Date;
+import java.util.UUID;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.io.fs.MatchResult;
+import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions;
+import org.apache.beam.sdk.io.fs.ResourceId;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation.Required;
+import org.apache.beam.sdk.testing.FileChecksumMatcher;
+import org.apache.beam.sdk.testing.RetryFailures;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+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.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * ValidatesRunner test for the support of {@link
+ * org.apache.beam.sdk.transforms.DoFn.RequiresStableInput} annotation.
+ */
+@RunWith(JUnit4.class)
+public class ParDoRequiresStableInputTest {
+
+ private static final String VALUE = "value";
+ // SHA-1 hash of string "value"
+ private static final String VALUE_CHECKSUM =
"f32b67c7e26342af42efabc674d441dca0a281c5";
+
+ private static class PairWithRandomKeyFn extends SimpleFunction<String,
KV<String, String>> {
+ @Override
+ public KV<String, String> apply(String value) {
+ String key = UUID.randomUUID().toString();
+ return KV.of(key, value);
+ }
+ }
+
+ private static class MakeSideEffectAndThenFailFn extends DoFn<KV<String,
String>, String> {
+ private final String outputPrefix;
+
+ private MakeSideEffectAndThenFailFn(String outputPrefix) {
+ this.outputPrefix = outputPrefix;
+ }
+
+ @RequiresStableInput
+ @ProcessElement
+ public void processElement(ProcessContext c) throws Exception {
+ MatchResult matchResult = FileSystems.match(outputPrefix + "*");
+ boolean firstTime = (matchResult.metadata().size() == 0);
+
+ KV<String, String> kv = c.element();
+ writeTextToFileSideEffect(kv.getValue(), outputPrefix + kv.getKey());
+ if (firstTime) {
+ throw new Exception("Deliberate failure: should happen only once.");
+ }
+ }
+
+ private static void writeTextToFileSideEffect(String text, String
filename) throws IOException {
+ ResourceId rid = FileSystems.matchNewResource(filename, false);
+ WritableByteChannel chan = FileSystems.create(rid, "text/plain");
+ chan.write(ByteBuffer.wrap(text.getBytes(Charset.defaultCharset())));
+ chan.close();
+ }
+ }
+
+ private static void
runRequiresStableInputPipeline(RequiresStableInputTestOptions options) {
+ Pipeline p = Pipeline.create(options);
+
+ PCollection<String> singleton = p.apply("CreatePCollectionOfOneValue",
Create.of(VALUE));
+ singleton
+ .apply("Single-PairWithRandomKey", MapElements.via(new
PairWithRandomKeyFn()))
+ .apply(
+ "Single-MakeSideEffectAndThenFail",
+ ParDo.of(new
MakeSideEffectAndThenFailFn(options.getParDoSingleOutputPrefix())));
+ singleton
+ .apply("Multi-PairWithRandomKey", MapElements.via(new
PairWithRandomKeyFn()))
+ .apply(
+ "Multi-MakeSideEffectAndThenFail",
+ ParDo.of(new
MakeSideEffectAndThenFailFn(options.getParDoMultiOutputPrefix()))
+ .withOutputTags(new TupleTag<>(), TupleTagList.empty()));
+
+ p.run().waitUntilFinish();
+ }
+
+ @BeforeClass
+ public static void setup() {
+ PipelineOptionsFactory.register(TestPipelineOptions.class);
+ }
+
+ /**
+ * Option for ParDoRequiresStableInputTest.
+ *
+ * <p>Define the output prefixes for tests on {@link ParDo.SingleOutput} and
{@link
+ * ParDo.MultiOutput}.
+ */
+ public interface RequiresStableInputTestOptions extends TestPipelineOptions {
Review comment:
I think this should be an integration test if it requires more access than
the current ValidatesRunner tests.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 138524)
Time Spent: 50m (was: 40m)
> Support annotating that a DoFn requires stable / deterministic input for
> replay/retry
> -------------------------------------------------------------------------------------
>
> Key: BEAM-3194
> URL: https://issues.apache.org/jira/browse/BEAM-3194
> Project: Beam
> Issue Type: New Feature
> Components: beam-model
> Reporter: Kenneth Knowles
> Assignee: Yueyang Qiu
> Priority: Major
> Time Spent: 50m
> Remaining Estimate: 0h
>
> See the thread:
> https://lists.apache.org/thread.html/5fd81ce371aeaf642665348f8e6940e308e04275dd7072f380f9f945@%3Cdev.beam.apache.org%3E
> We need this in order to have truly cross-runner end-to-end exactly once via
> replay + idempotence.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)