[ https://issues.apache.org/jira/browse/BEAM-3194?focusedWorklogId=135942&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-135942 ]
ASF GitHub Bot logged work on BEAM-3194: ---------------------------------------- Author: ASF GitHub Bot Created on: 18/Aug/18 00:41 Start Date: 18/Aug/18 00:41 Worklog Time Spent: 10m Work Description: lukecwik 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_r211058610 ########## 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 believe we should treat this as an IT instead of a regular ValidatesRunner test because it relies on external state **OR** you can keep it as a ValidatesRunner test by having the test rely on AttemptedMetrics since it is the only thing that you can modify that is part of the Apache Beam model and also changes even if a bundle fails. Using this would allow you to avoid files (which is a hassle in testing (who cleans them up, what if the IO system is down, requires extra configuration from a user)) and then you could keep this test as a ValidatesRunner test which is quite valuable. It would require runners to support attempted metrics in addition to requires stable input to test the feature though. ---------------------------------------------------------------- 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: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 135942) Time Spent: 0.5h (was: 20m) > 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: 0.5h > 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)