Repository: beam Updated Branches: refs/heads/master 697063297 -> 2dc61f637
Remove GetProducingTransformInternal This isn't part of the PValue API, so remove from POutputValueBase. Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/3d53b72c Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/3d53b72c Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/3d53b72c Branch: refs/heads/master Commit: 3d53b72c9a23705e55ec0a802c81b756d113e601 Parents: 9beb04e Author: Thomas Groh <tg...@google.com> Authored: Mon Apr 3 16:26:10 2017 -0700 Committer: Thomas Groh <tg...@google.com> Committed: Mon Apr 3 16:29:15 2017 -0700 ---------------------------------------------------------------------- .../beam/sdk/values/POutputValueBase.java | 41 +++----------------- 1 file changed, 6 insertions(+), 35 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/3d53b72c/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutputValueBase.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutputValueBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutputValueBase.java index cdef58c..5bd424d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutputValueBase.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutputValueBase.java @@ -53,40 +53,16 @@ public abstract class POutputValueBase implements POutput { } /** - * Returns the {@link AppliedPTransform} that this {@link POutputValueBase} - * is an output of. + * Records that this {@link POutputValueBase} is an output with the given name of the given {@link + * AppliedPTransform}. * - * <p>For internal use only. - */ - public AppliedPTransform<?, ?, ?> getProducingTransformInternal() { - return producingTransform; - } - - /** - * Records that this {@link POutputValueBase} is an output with the - * given name of the given {@link AppliedPTransform}. + * <p>By default, does nothing. * - * <p>To be invoked only by {@link POutput#recordAsOutput} - * implementations. Not to be invoked directly by user code. + * <p>To be invoked only by {@link POutput#recordAsOutput} implementations. Not to be invoked + * directly by user code. */ @Override - public void recordAsOutput(AppliedPTransform<?, ?, ?> transform) { - if (producingTransform != null) { - // Already used this POutput as a PTransform output. This can - // happen if the POutput is an output of a transform within a - // composite transform, and is also the result of the composite. - // We want to record the "immediate" atomic transform producing - // this output, and ignore all later composite transforms that - // also produce this output. - // - // Pipeline.applyInternal() uses !hasProducingTransform() to - // avoid calling this operation redundantly, but - // hasProducingTransform() doesn't apply to POutputValueBases - // that aren't PValues or composites of PValues, e.g., PDone. - return; - } - producingTransform = transform; - } + public void recordAsOutput(AppliedPTransform<?, ?, ?> transform) {} /** * Default behavior for {@link #finishSpecifyingOutput(PInput, PTransform)}} is @@ -95,9 +71,4 @@ public abstract class POutputValueBase implements POutput { */ @Override public void finishSpecifyingOutput(PInput input, PTransform<?, ?> transform) { } - - /** - * The {@link PTransform} that produces this {@link POutputValueBase}. - */ - private AppliedPTransform<?, ?, ?> producingTransform; }