[ 
https://issues.apache.org/jira/browse/BEAM-2929?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16295674#comment-16295674
 ] 

ASF GitHub Bot commented on BEAM-2929:
--------------------------------------

lukecwik closed pull request #4251: [BEAM-2929] Remove Dataflow expansions for 
PCollectionView that have been migrated into the Dataflow service for the 
portability framework.
URL: https://github.com/apache/beam/pull/4251
 
 
   

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/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
index 4f9b939eaf3..5c26e0da0e1 100644
--- 
a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
+++ 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
@@ -21,6 +21,7 @@
 import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
 import static com.google.common.base.Strings.isNullOrEmpty;
+import static org.apache.beam.runners.dataflow.DataflowRunner.hasExperiment;
 import static org.apache.beam.runners.dataflow.util.Structs.addBoolean;
 import static org.apache.beam.runners.dataflow.util.Structs.addDictionary;
 import static org.apache.beam.runners.dataflow.util.Structs.addList;
@@ -84,6 +85,7 @@
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.HasDisplayData;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
@@ -349,7 +351,7 @@ public Job translate(List<DataflowPackage> packages) {
         job.setLabels(options.getLabels());
       }
       if (options.isStreaming()
-          && !DataflowRunner.hasExperiment(options, 
"enable_windmill_service")) {
+          && !hasExperiment(options, "enable_windmill_service")) {
         // Use separate data disk for streaming.
         Disk disk = new Disk();
         disk.setDiskType(options.getWorkerDiskType());
@@ -447,13 +449,22 @@ public void 
visitPrimitiveTransform(TransformHierarchy.Node node) {
     public void visitValue(PValue value, TransformHierarchy.Node producer) {
       LOG.debug("Checking translation of {}", value);
       // Primitive transforms are the only ones assigned step names.
-      if (producer.getTransform() instanceof CreateDataflowView) {
-        // CreateDataflowView produces a dummy output (as it must be a 
primitive transform) but
-        // in the Dataflow Job graph produces only the view and not the output 
PCollection.
+      if (producer.getTransform() instanceof CreateDataflowView
+          && !hasExperiment(options, "beam_fn_api")) {
+        // CreateDataflowView produces a dummy output (as it must be a 
primitive transform)
+        // but in the Dataflow Job graph produces only the view and not the 
output PCollection.
         asOutputReference(
             ((CreateDataflowView) producer.getTransform()).getView(),
             producer.toAppliedPTransform(getPipeline()));
         return;
+      } else if (producer.getTransform() instanceof View.CreatePCollectionView
+          && hasExperiment(options, "beam_fn_api")) {
+        // View.CreatePCollectionView produces a dummy output (as it must be a 
primitive transform)
+        // but in the Dataflow Job graph produces only the view and not the 
output PCollection.
+        asOutputReference(
+            ((View.CreatePCollectionView) producer.getTransform()).getView(),
+            producer.toAppliedPTransform(getPipeline()));
+        return;
       }
       asOutputReference(value, producer.toAppliedPTransform(getPipeline()));
     }
@@ -679,6 +690,24 @@ public String toString() {
   ///////////////////////////////////////////////////////////////////////////
 
   static {
+    registerTransformTranslator(
+        View.CreatePCollectionView.class,
+        new TransformTranslator<View.CreatePCollectionView>() {
+          @Override
+          public void translate(View.CreatePCollectionView transform, 
TranslationContext context) {
+            translateTyped(transform, context);
+          }
+
+          private <ElemT, ViewT> void translateTyped(
+              View.CreatePCollectionView<ElemT, ViewT> transform, 
TranslationContext context) {
+            StepTranslationContext stepContext =
+                context.addStep(transform, "CollectionToSingleton");
+            PCollection<ElemT> input = context.getInput(transform);
+            stepContext.addInput(PropertyNames.PARALLEL_INPUT, input);
+            stepContext.addCollectionToSingletonOutput(input, 
transform.getView());
+          }
+        });
+
     registerTransformTranslator(
         CreateDataflowView.class,
         new TransformTranslator<CreateDataflowView>() {
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 ddad43fe6ec..942d36b3e63 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
@@ -382,11 +382,13 @@ public static DataflowRunner fromOptions(PipelineOptions 
options) {
           .add(
               PTransformOverride.of(
                   PTransformMatchers.classEqualTo(Read.Unbounded.class),
-                  new StreamingUnboundedReadOverrideFactory()))
-          .add(
-              PTransformOverride.of(
-                  
PTransformMatchers.classEqualTo(View.CreatePCollectionView.class),
-                  new StreamingCreatePCollectionViewFactory()));
+                  new StreamingUnboundedReadOverrideFactory()));
+      if (!hasExperiment(options, "beam_fn_api")) {
+        overridesBuilder.add(
+            PTransformOverride.of(
+                
PTransformMatchers.classEqualTo(View.CreatePCollectionView.class),
+                new StreamingCreatePCollectionViewFactory()));
+      }
     } else {
       overridesBuilder
           // State and timer pardos are implemented by expansion to 
GBK-then-ParDo
@@ -397,7 +399,9 @@ public static DataflowRunner fromOptions(PipelineOptions 
options) {
           .add(
               PTransformOverride.of(
                   PTransformMatchers.stateOrTimerParDoSingle(),
-                  
BatchStatefulParDoOverrides.singleOutputOverrideFactory(options)))
+                  
BatchStatefulParDoOverrides.singleOutputOverrideFactory(options)));
+      if (!hasExperiment(options, "beam_fn_api")) {
+        overridesBuilder
           .add(
               PTransformOverride.of(
                   PTransformMatchers.classEqualTo(View.AsMap.class),
@@ -422,6 +426,7 @@ public static DataflowRunner fromOptions(PipelineOptions 
options) {
                   PTransformMatchers.classEqualTo(View.AsIterable.class),
                   new ReflectiveViewOverrideFactory(
                       BatchViewOverrides.BatchViewAsIterable.class, this)));
+      }
     }
     // Expands into Reshuffle and single-output ParDo, so has to be before the 
overrides below.
     if (hasExperiment(options, "beam_fn_api")) {


 

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


> Dataflow support for portable side input
> ----------------------------------------
>
>                 Key: BEAM-2929
>                 URL: https://issues.apache.org/jira/browse/BEAM-2929
>             Project: Beam
>          Issue Type: Sub-task
>          Components: runner-dataflow
>            Reporter: Henning Rohde
>            Assignee: Luke Cwik
>              Labels: portability
>




--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to