[ 
https://issues.apache.org/jira/browse/BEAM-4684?focusedWorklogId=118877&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-118877
 ]

ASF GitHub Bot logged work on BEAM-4684:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 03/Jul/18 22:26
            Start Date: 03/Jul/18 22:26
    Worklog Time Spent: 10m 
      Work Description: kennknowles closed pull request #5787: [BEAM-4684] 
Support @RequiresStableInput on Dataflow runner in Java SDK
URL: https://github.com/apache/beam/pull/5787
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformMatchers.java
 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformMatchers.java
index e44c20fe9bb..0a085b7f2a2 100644
--- 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformMatchers.java
+++ 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformMatchers.java
@@ -104,6 +104,64 @@ public String toString() {
     }
   }
 
+  /**
+   * A {@link PTransformMatcher} that matches a {@link ParDo.SingleOutput} 
containing a {@link DoFn}
+   * that requires stable input, as signified by {@link 
ProcessElementMethod#requiresStableInput()}.
+   */
+  public static PTransformMatcher requiresStableInputParDoSingle() {
+    return new PTransformMatcher() {
+      @Override
+      public boolean matches(AppliedPTransform<?, ?, ?> application) {
+        PTransform<?, ?> transform = application.getTransform();
+        if (transform instanceof ParDo.SingleOutput) {
+          DoFn<?, ?> fn = ((ParDo.SingleOutput<?, ?>) transform).getFn();
+          DoFnSignature signature = DoFnSignatures.signatureForDoFn(fn);
+          return signature.processElement().requiresStableInput();
+        }
+        return false;
+      }
+
+      @Override
+      public boolean matchesDuringValidation(AppliedPTransform<?, ?, ?> 
application) {
+        return false;
+      }
+
+      @Override
+      public String toString() {
+        return 
MoreObjects.toStringHelper("RequiresStableInputParDoSingleMatcher").toString();
+      }
+    };
+  }
+
+  /**
+   * A {@link PTransformMatcher} that matches a {@link ParDo.MultiOutput} 
containing a {@link DoFn}
+   * that requires stable input, as signified by {@link 
ProcessElementMethod#requiresStableInput()}.
+   */
+  public static PTransformMatcher requiresStableInputParDoMulti() {
+    return new PTransformMatcher() {
+      @Override
+      public boolean matches(AppliedPTransform<?, ?, ?> application) {
+        PTransform<?, ?> transform = application.getTransform();
+        if (transform instanceof ParDo.MultiOutput) {
+          DoFn<?, ?> fn = ((ParDo.MultiOutput<?, ?>) transform).getFn();
+          DoFnSignature signature = DoFnSignatures.signatureForDoFn(fn);
+          return signature.processElement().requiresStableInput();
+        }
+        return false;
+      }
+
+      @Override
+      public boolean matchesDuringValidation(AppliedPTransform<?, ?, ?> 
application) {
+        return false;
+      }
+
+      @Override
+      public String toString() {
+        return 
MoreObjects.toStringHelper("RequiresStableInputParDoMultiMatcher").toString();
+      }
+    };
+  }
+
   /**
    * A {@link PTransformMatcher} that matches a {@link ParDo.SingleOutput} 
containing a {@link DoFn}
    * that is splittable, as signified by {@link 
ProcessElementMethod#isSplittable()}.
diff --git 
a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java
 
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java
index 7d995aecd21..5211f9281dc 100644
--- 
a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java
+++ 
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java
@@ -322,6 +322,33 @@ public void parDoMultiWithTimers() {
     
assertThat(PTransformMatchers.stateOrTimerParDoSingle().matches(parDoApplication),
 is(false));
   }
 
+  @Test
+  public void parDoRequiresStableInput() {
+    DoFn<Object, Object> doFnRSI =
+        new DoFn<Object, Object>() {
+          @RequiresStableInput
+          @ProcessElement
+          public void process(ProcessContext ctxt) {}
+        };
+
+    AppliedPTransform<?, ?, ?> single = getAppliedTransform(ParDo.of(doFn));
+    AppliedPTransform<?, ?, ?> singleRSI = 
getAppliedTransform(ParDo.of(doFnRSI));
+    AppliedPTransform<?, ?, ?> multi =
+        getAppliedTransform(ParDo.of(doFn).withOutputTags(new TupleTag<>(), 
TupleTagList.empty()));
+    AppliedPTransform<?, ?, ?> multiRSI =
+        getAppliedTransform(
+            ParDo.of(doFnRSI).withOutputTags(new TupleTag<>(), 
TupleTagList.empty()));
+
+    
assertThat(PTransformMatchers.requiresStableInputParDoSingle().matches(single), 
is(false));
+    
assertThat(PTransformMatchers.requiresStableInputParDoSingle().matches(singleRSI),
 is(true));
+    
assertThat(PTransformMatchers.requiresStableInputParDoSingle().matches(multi), 
is(false));
+    
assertThat(PTransformMatchers.requiresStableInputParDoSingle().matches(multiRSI),
 is(false));
+    
assertThat(PTransformMatchers.requiresStableInputParDoMulti().matches(single), 
is(false));
+    
assertThat(PTransformMatchers.requiresStableInputParDoMulti().matches(singleRSI),
 is(false));
+    
assertThat(PTransformMatchers.requiresStableInputParDoMulti().matches(multi), 
is(false));
+    
assertThat(PTransformMatchers.requiresStableInputParDoMulti().matches(multiRSI),
 is(true));
+  }
+
   @Test
   public void parDoWithFnTypeWithMatchingType() {
     DoFn<Object, Object> fn =
diff --git 
a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index 6f10f0478c9..dedbfee95b0 100644
--- 
a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -443,6 +443,16 @@ protected DataflowRunner(DataflowPipelineOptions options) {
                         BatchViewOverrides.BatchViewAsIterable.class, this)));
       }
     }
+    // Uses Reshuffle, so has to be before the Reshuffle override
+    overridesBuilder.add(
+        PTransformOverride.of(
+            PTransformMatchers.requiresStableInputParDoSingle(),
+            RequiresStableInputParDoOverrides.singleOutputOverrideFactory()));
+    // Uses Reshuffle, so has to be before the Reshuffle override
+    overridesBuilder.add(
+        PTransformOverride.of(
+            PTransformMatchers.requiresStableInputParDoMulti(),
+            RequiresStableInputParDoOverrides.multiOutputOverrideFactory()));
     // Expands into Reshuffle and single-output ParDo, so has to be before the 
overrides below.
     if (hasExperiment(options, "beam_fn_api")) {
       overridesBuilder.add(
diff --git 
a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RequiresStableInputParDoOverrides.java
 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RequiresStableInputParDoOverrides.java
new file mode 100644
index 00000000000..31b5520498e
--- /dev/null
+++ 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RequiresStableInputParDoOverrides.java
@@ -0,0 +1,113 @@
+/*
+ * 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.runners.dataflow;
+
+import java.util.Map;
+import org.apache.beam.runners.core.construction.PTransformReplacements;
+import org.apache.beam.runners.core.construction.ReplacementOutputs;
+import org.apache.beam.sdk.runners.AppliedPTransform;
+import org.apache.beam.sdk.runners.PTransformOverrideFactory;
+import org.apache.beam.sdk.transforms.DoFn.RequiresStableInput;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Reshuffle;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+
+/** Transform overrides for supporting {@link RequiresStableInput} in the 
Dataflow runner. */
+class RequiresStableInputParDoOverrides {
+
+  /**
+   * Returns a {@link PTransformOverrideFactory} that inserts a {@link 
Reshuffle.ViaRandomKey}
+   * before a {@link ParDo.SingleOutput} that uses the {@link 
RequiresStableInput} annotation.
+   */
+  static <InputT, OutputT>
+      PTransformOverrideFactory<
+              PCollection<InputT>, PCollection<OutputT>, 
ParDo.SingleOutput<InputT, OutputT>>
+          singleOutputOverrideFactory() {
+    return new SingleOutputOverrideFactory<>();
+  }
+
+  /**
+   * Returns a {@link PTransformOverrideFactory} that inserts a {@link 
Reshuffle.ViaRandomKey}
+   * before a {@link ParDo.MultiOutput} that uses the {@link 
RequiresStableInput} annotation.
+   */
+  static <InputT, OutputT>
+      PTransformOverrideFactory<
+              PCollection<InputT>, PCollectionTuple, ParDo.MultiOutput<InputT, 
OutputT>>
+          multiOutputOverrideFactory() {
+    return new MultiOutputOverrideFactory<>();
+  }
+
+  private static class SingleOutputOverrideFactory<InputT, OutputT>
+      implements PTransformOverrideFactory<
+          PCollection<InputT>, PCollection<OutputT>, 
ParDo.SingleOutput<InputT, OutputT>> {
+
+    @Override
+    public PTransformReplacement<PCollection<InputT>, PCollection<OutputT>> 
getReplacementTransform(
+        AppliedPTransform<
+                PCollection<InputT>, PCollection<OutputT>, 
ParDo.SingleOutput<InputT, OutputT>>
+            appliedTransform) {
+      return PTransformReplacement.of(
+          PTransformReplacements.getSingletonMainInput(appliedTransform),
+          new PTransform<PCollection<InputT>, PCollection<OutputT>>() {
+            @Override
+            public PCollection<OutputT> expand(PCollection<InputT> input) {
+              return input
+                  .apply("Materialize input", Reshuffle.viaRandomKey())
+                  .apply("ParDo with stable input", 
appliedTransform.getTransform());
+            }
+          });
+    }
+
+    @Override
+    public Map<PValue, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PValue> outputs, PCollection<OutputT> newOutput) {
+      return ReplacementOutputs.singleton(outputs, newOutput);
+    }
+  }
+
+  private static class MultiOutputOverrideFactory<InputT, OutputT>
+      implements PTransformOverrideFactory<
+          PCollection<InputT>, PCollectionTuple, ParDo.MultiOutput<InputT, 
OutputT>> {
+
+    @Override
+    public PTransformReplacement<PCollection<InputT>, PCollectionTuple> 
getReplacementTransform(
+        AppliedPTransform<PCollection<InputT>, PCollectionTuple, 
ParDo.MultiOutput<InputT, OutputT>>
+            appliedTransform) {
+      return PTransformReplacement.of(
+          PTransformReplacements.getSingletonMainInput(appliedTransform),
+          new PTransform<PCollection<InputT>, PCollectionTuple>() {
+            @Override
+            public PCollectionTuple expand(PCollection<InputT> input) {
+              return input
+                  .apply("Materialize input", Reshuffle.viaRandomKey())
+                  .apply("ParDo with stable input", 
appliedTransform.getTransform());
+            }
+          });
+    }
+
+    @Override
+    public Map<PValue, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PValue> outputs, PCollectionTuple newOutput) {
+      return ReplacementOutputs.tagged(outputs, newOutput);
+    }
+  }
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
index f749f1e1d8f..7bd3f0ec40a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
@@ -212,16 +212,13 @@ private void checkNoMoreMatches(final 
List<PTransformOverride> overrides) {
           @Override
           public CompositeBehavior enterCompositeTransform(Node node) {
             if (!node.isRootNode()) {
-              for (PTransformOverride override : overrides) {
-                if 
(override.getMatcher().matches(node.toAppliedPTransform(getPipeline()))) {
-                  matched.put(node, override);
-                }
-              }
+              checkForMatches(node);
             }
-            if (!matched.containsKey(node)) {
+            if (matched.containsKey(node)) {
+              return CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
+            } else {
               return CompositeBehavior.ENTER_TRANSFORM;
             }
-            return CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
           }
 
           @Override
@@ -234,8 +231,14 @@ public void leaveCompositeTransform(Node node) {
 
           @Override
           public void visitPrimitiveTransform(Node node) {
+            checkForMatches(node);
+          }
+
+          private void checkForMatches(Node node) {
             for (PTransformOverride override : overrides) {
-              if 
(override.getMatcher().matches(node.toAppliedPTransform(getPipeline()))) {
+              if (override
+                  .getMatcher()
+                  
.matchesDuringValidation(node.toAppliedPTransform(getPipeline()))) {
                 matched.put(node, override);
               }
             }
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformMatcher.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformMatcher.java
index 9821596052d..ba8541b2b7a 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformMatcher.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformMatcher.java
@@ -33,6 +33,17 @@
 public interface PTransformMatcher {
   boolean matches(AppliedPTransform<?, ?, ?> application);
 
+  /**
+   * An {@link AppliedPTransform} matched by a {@link PTransformMatcher} will 
be replaced during
+   * pipeline surgery, and is often expected to be gone the new pipeline. For 
the {@link
+   * AppliedPTransform} that is expected to remain in the pipeline after 
surgery, the corresponding
+   * {@link PTransformMatcher} should override this method, such that it will 
not be matched during
+   * the validation.
+   */
+  default boolean matchesDuringValidation(AppliedPTransform<?, ?, ?> 
application) {
+    return matches(application);
+  }
+
   default PTransformMatcher and(PTransformMatcher matcher) {
     return application -> this.matches(application) && 
matcher.matches(application);
   }


 

----------------------------------------------------------------
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: 118877)
    Time Spent: 2h 40m  (was: 2.5h)

> Support @RequiresStableInput on Dataflow runner in Java SDK
> -----------------------------------------------------------
>
>                 Key: BEAM-4684
>                 URL: https://issues.apache.org/jira/browse/BEAM-4684
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-java-core
>            Reporter: Yueyang Qiu
>            Assignee: Yueyang Qiu
>            Priority: Major
>          Time Spent: 2h 40m
>  Remaining Estimate: 0h
>
> https://docs.google.com/document/d/117yRKbbcEdm3eIKB_26BHOJGmHSZl1YNoF0RqWGtqAM



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to