You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ke...@apache.org on 2017/06/30 21:31:15 UTC

[13/50] beam git commit: Use PCollectionViews.toAdditionalInputs in ParDo

Use PCollectionViews.toAdditionalInputs in ParDo


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

Branch: refs/heads/gearpump-runner
Commit: ed476dd2807577c8069087aa0764b21d1bb06512
Parents: 4238276
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Jun 27 14:41:30 2017 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Jun 27 21:08:11 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/transforms/ParDo.java    | 14 +++-----------
 1 file changed, 3 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/ed476dd2/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
index edf1419..db1f791 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
@@ -20,7 +20,6 @@ package org.apache.beam.sdk.transforms;
 import static com.google.common.base.Preconditions.checkArgument;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import java.io.Serializable;
 import java.lang.reflect.ParameterizedType;
 import java.lang.reflect.Type;
@@ -50,6 +49,7 @@ import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
 import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PCollectionViews;
 import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
@@ -662,11 +662,7 @@ public class ParDo {
      */
     @Override
     public Map<TupleTag<?>, PValue> getAdditionalInputs() {
-      ImmutableMap.Builder<TupleTag<?>, PValue> additionalInputs = ImmutableMap.builder();
-      for (PCollectionView<?> sideInput : sideInputs) {
-        additionalInputs.put(sideInput.getTagInternal(), sideInput.getPCollection());
-      }
-      return additionalInputs.build();
+      return PCollectionViews.toAdditionalInputs(sideInputs);
     }
   }
 
@@ -807,11 +803,7 @@ public class ParDo {
      */
     @Override
     public Map<TupleTag<?>, PValue> getAdditionalInputs() {
-      ImmutableMap.Builder<TupleTag<?>, PValue> additionalInputs = ImmutableMap.builder();
-      for (PCollectionView<?> sideInput : sideInputs) {
-        additionalInputs.put(sideInput.getTagInternal(), sideInput.getPCollection());
-      }
-      return additionalInputs.build();
+      return PCollectionViews.toAdditionalInputs(sideInputs);
     }
   }