From 8f80c0d232b30e70b45e68aad8d1233262dfcfc8 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 9 May 2017 18:32:14 -0700 Subject: [PATCH] Mark PValue and PValueBase Internal These should not be referred to as their abstract types by users, and PipelineRunners won't interact with PValues that aren't PCollections or PCollectionViews. --- .../java/org/apache/beam/sdk/values/PValue.java | 12 +++++++----- .../org/apache/beam/sdk/values/PValueBase.java | 15 +++++---------- 2 files changed, 12 insertions(+), 15 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java index d9f692014d86..108902879236 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java @@ -18,13 +18,15 @@ package org.apache.beam.sdk.values; import java.util.Map; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.transforms.PTransform; /** - * The interface for values that can be input to and output from {@link PTransform PTransforms}. + * For internal use. No backwards compatibility guarantees. * - *

It is recommended to extend {@link PValueBase} + *

A primitive value within Beam. */ +@Internal public interface PValue extends POutput, PInput { /** @@ -45,13 +47,13 @@ public interface PValue extends POutput, PInput { * After building, finalizes this {@code PValue} to make it ready for being used as an input to a * {@link org.apache.beam.sdk.transforms.PTransform}. * - *

Automatically invoked whenever {@code apply()} is invoked on this {@code PValue}, after - * {@link PValue#finishSpecifying(PInput, PTransform)} has been called on each component {@link - * PValue}, so users do not normally call this explicitly. + *

Automatically invoked whenever {@code apply()} is invoked on this {@code PValue}. Users + * should not normally call this explicitly. * * @param upstreamInput the {@link PInput} the {@link PTransform} was applied to to produce this * output * @param upstreamTransform the {@link PTransform} that produced this {@link PValue} */ + @Internal void finishSpecifying(PInput upstreamInput, PTransform upstreamTransform); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java index 4de0589d1222..6f638d7b9a3a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java @@ -22,22 +22,17 @@ import java.util.Collections; import java.util.Map; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.NameUtils; /** - * A {@link PValueBase} is an abstract base class that provides - * sensible default implementations for methods of {@link PValue}. - * In particular, this includes functionality for getting/setting: + * For internal use. No backwards compatibility guarantees. * - *

- * - *

For internal use. + *

An abstract base class that provides default implementations for some methods of + * {@link PValue}. */ +@Internal public abstract class PValueBase implements PValue { private final Pipeline pipeline;