Repository: beam
Updated Branches:
  refs/heads/master 41de9301a -> b65db428a


Remove uses of CreatePCollectionView#getView

Views output by this transform should be obtained by inspecting the
graph node, not by interrogating the PTransform. Doing otherwise may use
incorrect views after Graph Surgery has been performed.

The result of getView can be used to, for example, return the same type
of view. The view returned by this method should be interpreted as a
PCollectionView spec rather than a PValue, as the graph containing the
PTransform and PCollectionView can change independently.


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/a8b46cf9
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/a8b46cf9
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/a8b46cf9

Branch: refs/heads/master
Commit: a8b46cf984181b6ec0dfc8a59c5e1ed180e2a29e
Parents: 41de930
Author: Thomas Groh <tg...@google.com>
Authored: Tue Feb 21 17:10:13 2017 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Wed Feb 22 12:42:22 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/beam/runners/apex/ApexRunner.java     |  4 ----
 .../runners/apex/translation/ApexPipelineTranslator.java  |  8 ++++----
 .../apache/beam/runners/direct/ViewEvaluatorFactory.java  |  1 +
 .../main/java/org/apache/beam/sdk/transforms/View.java    | 10 ++++++++++
 4 files changed, 15 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/a8b46cf9/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java
----------------------------------------------------------------------
diff --git 
a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java 
b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java
index e220e6c..1eb5e72 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java
@@ -213,10 +213,6 @@ public class ApexRunner extends 
PipelineRunner<ApexRunnerResult> {
       return new CreateApexPCollectionView<>(view);
     }
 
-    public PCollectionView<ViewT> getView() {
-      return view;
-    }
-
     @Override
     public PCollectionView<ViewT> expand(PCollection<List<ElemT>> input) {
       return view;

http://git-wip-us.apache.org/repos/asf/beam/blob/a8b46cf9/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java
----------------------------------------------------------------------
diff --git 
a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java
 
b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java
index c8e0290..0818c36 100644
--- 
a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java
+++ 
b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java
@@ -157,7 +157,7 @@ public class ApexPipelineTranslator implements 
Pipeline.PipelineVisitor {
     @Override
     public void translate(CreateApexPCollectionView<ElemT, ViewT> transform,
         TranslationContext context) {
-      PCollectionView<ViewT> view = transform.getView();
+      PCollectionView<ViewT> view = (PCollectionView<ViewT>) 
context.getOutput();
       context.addView(view);
       LOG.debug("view {}", view.getName());
     }
@@ -168,9 +168,9 @@ public class ApexPipelineTranslator implements 
Pipeline.PipelineVisitor {
     private static final long serialVersionUID = 1L;
 
     @Override
-    public void translate(CreatePCollectionView<ElemT, ViewT> transform,
-        TranslationContext context) {
-      PCollectionView<ViewT> view = transform.getView();
+    public void translate(
+        CreatePCollectionView<ElemT, ViewT> transform, TranslationContext 
context) {
+      PCollectionView<ViewT> view = (PCollectionView<ViewT>) 
context.getOutput();
       context.addView(view);
       LOG.debug("view {}", view.getName());
     }

http://git-wip-us.apache.org/repos/asf/beam/blob/a8b46cf9/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java
----------------------------------------------------------------------
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java
index 0fa6254..1548772 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java
@@ -148,6 +148,7 @@ class ViewEvaluatorFactory implements 
TransformEvaluatorFactory {
     }
 
     @Override
+    @SuppressWarnings("deprecation")
     public PCollectionView<ViewT> expand(PCollection<Iterable<ElemT>> input) {
       return og.getView();
     }

http://git-wip-us.apache.org/repos/asf/beam/blob/a8b46cf9/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
----------------------------------------------------------------------
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
index d18a0c6..1986ac5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
@@ -20,6 +20,7 @@ package org.apache.beam.sdk.transforms;
 import java.util.List;
 import java.util.Map;
 import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.util.PCollectionViews;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
@@ -434,6 +435,15 @@ public class View {
       return new CreatePCollectionView<>(view);
     }
 
+    /**
+     * Return the {@link PCollectionView} that is returned by applying this 
{@link PTransform}.
+     *
+     * <p>This should not be used to obtain the output of any given 
application of this
+     * {@link PTransform}. That should be obtained by inspecting the {@link 
Node}
+     * that contains this {@link CreatePCollectionView}, as this view may have 
been replaced within
+     * pipeline surgery.
+     */
+    @Deprecated
     public PCollectionView<ViewT> getView() {
       return view;
     }

Reply via email to