beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From k...@apache.org
Subject [1/2] beam git commit: Remove dead (and wrong) viewFromProto overload
Date Tue, 11 Jul 2017 21:44:10 GMT
Repository: beam
Updated Branches:
  refs/heads/master a22f1a05a -> 011e2796d


Remove dead (and wrong) viewFromProto overload


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

Branch: refs/heads/master
Commit: aaffe15ee891b5656e448eac4bd3a7ff72eee315
Parents: 138641f
Author: Kenneth Knowles <klk@google.com>
Authored: Tue Jul 11 10:09:12 2017 -0700
Committer: Kenneth Knowles <klk@google.com>
Committed: Tue Jul 11 14:11:39 2017 -0700

----------------------------------------------------------------------
 .../core/construction/ParDoTranslation.java     | 21 --------------------
 .../core/construction/ParDoTranslationTest.java |  2 +-
 2 files changed, 1 insertion(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/aaffe15e/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
----------------------------------------------------------------------
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
index 90c9aad..03f29ff 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
@@ -41,7 +41,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.IterableCoder;
 import org.apache.beam.sdk.common.runner.v1.RunnerApi;
@@ -509,26 +508,6 @@ public class ParDoTranslation {
     return builder.build();
   }
 
-  public static PCollectionView<?> viewFromProto(
-      Pipeline pipeline,
-      SideInput sideInput,
-      String localName,
-      RunnerApi.PTransform parDoTransform,
-      Components components)
-      throws IOException {
-
-    String pCollectionId = parDoTransform.getInputsOrThrow(localName);
-
-    // This may be a PCollection defined in another language, but we should be
-    // able to rehydrate it enough to stick it in a side input. The coder may not
-    // be grokkable in Java.
-    PCollection<?> pCollection =
-        PCollectionTranslation.fromProto(
-            pipeline, components.getPcollectionsOrThrow(pCollectionId), components);
-
-    return viewFromProto(sideInput, localName, pCollection, parDoTransform, components);
-  }
-
   /**
    * Create a {@link PCollectionView} from a side input spec and an already-deserialized
{@link
    * PCollection} that should be wired up.

http://git-wip-us.apache.org/repos/asf/beam/blob/aaffe15e/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java
----------------------------------------------------------------------
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java
index 6fdf9d6..a87a16d 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java
@@ -162,9 +162,9 @@ public class ParDoTranslationTest {
         SideInput sideInput = parDoPayload.getSideInputsOrThrow(view.getTagInternal().getId());
         PCollectionView<?> restoredView =
             ParDoTranslation.viewFromProto(
-                rehydratedPipeline,
                 sideInput,
                 view.getTagInternal().getId(),
+                view.getPCollection(),
                 protoTransform,
                 protoComponents);
         assertThat(restoredView.getTagInternal(), equalTo(view.getTagInternal()));


Mime
View raw message