Repository: beam Updated Branches: refs/heads/master 7fde976d1 -> a6f460fe3
[BEAM-2306] Add checkstyle check to fail the build when @Deprecated is used without @deprecated javadoc (or vice versa). The check is disabled for existing violations where reason for deprecation and/or alternative is not clear. Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/d2901145 Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/d2901145 Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/d2901145 Branch: refs/heads/master Commit: d290114549c0b379774dbabe119a79d3ee1b2b56 Parents: 7fde976 Author: Alex Filatov <[email protected]> Authored: Mon Jul 10 13:20:49 2017 +0300 Committer: Kenneth Knowles <[email protected]> Committed: Wed Jul 19 09:03:31 2017 -0700 ---------------------------------------------------------------------- .../construction/CreatePCollectionViewTranslation.java | 11 ++++++++++- .../core/construction/PTransformTranslation.java | 4 ++++ .../beam/runners/core/InMemoryTimerInternals.java | 9 +++++++++ .../java/org/apache/beam/runners/core/StateTags.java | 3 +++ .../beam/runners/direct/DirectTimerInternals.java | 9 +++++++++ .../translation/wrappers/streaming/DoFnOperator.java | 9 +++++++++ .../apache/beam/runners/dataflow/DataflowRunner.java | 3 ++- .../options/DataflowPipelineWorkerPoolOptions.java | 3 +++ .../build-tools/src/main/resources/beam/checkstyle.xml | 8 ++++++++ .../src/main/java/org/apache/beam/sdk/coders/Coder.java | 12 +++++++++++- .../java/org/apache/beam/sdk/coders/CoderRegistry.java | 9 +++++++++ .../main/java/org/apache/beam/sdk/io/AvroSource.java | 6 ------ .../main/java/org/apache/beam/sdk/testing/PAssert.java | 5 +++-- .../java/org/apache/beam/sdk/testing/StreamingIT.java | 4 ++++ .../java/org/apache/beam/sdk/transforms/Combine.java | 1 - .../main/java/org/apache/beam/sdk/transforms/DoFn.java | 3 +++ .../main/java/org/apache/beam/sdk/transforms/View.java | 2 +- .../beam/sdk/transforms/reflect/DoFnInvokers.java | 9 --------- .../java/org/apache/beam/sdk/util/IdentityWindowFn.java | 1 - .../org/apache/beam/sdk/values/PCollectionViews.java | 1 - .../main/java/org/apache/beam/sdk/values/PValue.java | 4 ++-- .../org/apache/beam/sdk/coders/DefaultCoderTest.java | 3 ++- .../org/apache/beam/fn/harness/BoundedSourceRunner.java | 6 +++--- 23 files changed, 95 insertions(+), 30 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java ---------------------------------------------------------------------- diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java index 8fc99b9..c67d688 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java @@ -86,6 +86,10 @@ public class CreatePCollectionViewTranslation { PCollectionView.class.getSimpleName()); } + /** + * @deprecated runners should move away from translating `CreatePCollectionView` and treat this + * as part of the translation for a `ParDo` side input. + */ @Deprecated static class CreatePCollectionViewTranslator implements TransformPayloadTranslator<View.CreatePCollectionView<?, ?>> { @@ -112,7 +116,12 @@ public class CreatePCollectionViewTranslation { } } - /** Registers {@link CreatePCollectionViewTranslator}. */ + /** + * Registers {@link CreatePCollectionViewTranslator}. + * + * @deprecated runners should move away from translating `CreatePCollectionView` and treat this + * as part of the translation for a `ParDo` side input. + */ @AutoService(TransformPayloadTranslatorRegistrar.class) @Deprecated public static class Registrar implements TransformPayloadTranslatorRegistrar { http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java ---------------------------------------------------------------------- diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java index bae7b05..0b4a2ab 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java @@ -55,6 +55,10 @@ public class PTransformTranslation { // Less well-known. And where shall these live? public static final String WRITE_FILES_TRANSFORM_URN = "urn:beam:transform:write_files:0.1"; + /** + * @deprecated runners should move away from translating `CreatePCollectionView` and treat this + * as part of the translation for a `ParDo` side input. + */ @Deprecated public static final String CREATE_VIEW_TRANSFORM_URN = "urn:beam:transform:create_view:v1"; http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java ---------------------------------------------------------------------- diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java index e68bb24..c29ea19 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java @@ -107,6 +107,9 @@ public class InMemoryTimerInternals implements TimerInternals { setTimer(TimerData.of(timerId, namespace, target, timeDomain)); } + /** + * @deprecated use {@link #setTimer(StateNamespace, String, Instant, TimeDomain)}. + */ @Deprecated @Override public void setTimer(TimerData timerData) { @@ -136,6 +139,9 @@ public class InMemoryTimerInternals implements TimerInternals { throw new UnsupportedOperationException("Canceling a timer by ID is not yet supported."); } + /** + * @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. + */ @Deprecated @Override public void deleteTimer(StateNamespace namespace, String timerId) { @@ -145,6 +151,9 @@ public class InMemoryTimerInternals implements TimerInternals { } } + /** + * @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. + */ @Deprecated @Override public void deleteTimer(TimerData timer) { http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java ---------------------------------------------------------------------- diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java index 53f9edc..a98f47d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java @@ -302,6 +302,9 @@ public class StateTags { this.spec = spec; } + /** + * @deprecated use {@link StateSpec#bind} method via {@link #getSpec} for now. + */ @Override @Deprecated public StateT bind(StateTag.StateBinder binder) { http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java ---------------------------------------------------------------------- diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java index a099368..7db12a4 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java @@ -52,6 +52,9 @@ class DirectTimerInternals implements TimerInternals { timerUpdateBuilder.setTimer(TimerData.of(timerId, namespace, target, timeDomain)); } + /** + * @deprecated use {@link #setTimer(StateNamespace, String, Instant, TimeDomain)}. + */ @Deprecated @Override public void setTimer(TimerData timerData) { @@ -63,12 +66,18 @@ class DirectTimerInternals implements TimerInternals { throw new UnsupportedOperationException("Canceling of timer by ID is not yet supported."); } + /** + * @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. + */ @Deprecated @Override public void deleteTimer(StateNamespace namespace, String timerId) { throw new UnsupportedOperationException("Canceling of timer by ID is not yet supported."); } + /** + * @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. + */ @Deprecated @Override public void deleteTimer(TimerData timerKey) { http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java ---------------------------------------------------------------------- diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 350f323..a80f7b6 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -752,6 +752,9 @@ public class DoFnOperator<InputT, OutputT> setTimer(TimerData.of(timerId, namespace, target, timeDomain)); } + /** + * @deprecated use {@link #setTimer(StateNamespace, String, Instant, TimeDomain)}. + */ @Deprecated @Override public void setTimer(TimerData timerKey) { @@ -770,6 +773,9 @@ public class DoFnOperator<InputT, OutputT> } } + /** + * @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. + */ @Deprecated @Override public void deleteTimer(StateNamespace namespace, String timerId) { @@ -783,6 +789,9 @@ public class DoFnOperator<InputT, OutputT> "Canceling of a timer by ID is not yet supported."); } + /** + * @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. + */ @Deprecated @Override public void deleteTimer(TimerData timerKey) { http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java ---------------------------------------------------------------------- diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 57a5ea5..7335ef7 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -82,6 +82,7 @@ import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.PipelineResult.State; import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; @@ -1293,7 +1294,7 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> { * A marker {@link DoFn} for writing the contents of a {@link PCollection} to a streaming * {@link PCollectionView} backend implementation. */ - @Deprecated + @Internal public static class StreamingPCollectionViewWriterFn<T> extends DoFn<Iterable<T>, T> { private final PCollectionView<?> view; private final Coder<T> dataCoder; http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java ---------------------------------------------------------------------- diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java index 00d2194..2239462 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java @@ -53,6 +53,9 @@ public interface DataflowPipelineWorkerPoolOptions extends PipelineOptions { /** Use numWorkers machines. Do not autoscale the worker pool. */ NONE("AUTOSCALING_ALGORITHM_NONE"), + /** + * @deprecated use {@link #THROUGHPUT_BASED}. + */ @Deprecated BASIC("AUTOSCALING_ALGORITHM_BASIC"), http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml ---------------------------------------------------------------------- diff --git a/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml b/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml index ebbaa7d..b2a74a7 100644 --- a/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml +++ b/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml @@ -81,6 +81,14 @@ page at http://checkstyle.sourceforge.net/config.html --> <!-- + ANNOTATIONS CHECKS + + --> + + <module name="MissingDeprecated" /> + + <!-- + IMPORT CHECKS --> http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java index edcc3a8..78a4a02 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java @@ -56,7 +56,13 @@ import org.apache.beam.sdk.values.TypeDescriptor; * @param <T> the type of values being encoded and decoded */ public abstract class Coder<T> implements Serializable { - /** The context in which encoding or decoding is being done. */ + /** + * The context in which encoding or decoding is being done. + * + * @deprecated to implement a coder, do not use any `Context`. Just implement only those abstract + * methods which do not accept a `Context` and leave the default implementations for methods + * accepting a `Context`. + */ @Deprecated @Experimental(Kind.CODER_CONTEXT) public static class Context { @@ -127,6 +133,8 @@ public abstract class Coder<T> implements Serializable { * @throws IOException if writing to the {@code OutputStream} fails * for some reason * @throws CoderException if the value could not be encoded for some reason + * + * @deprecated only implement and call {@link #encode(Object value, OutputStream)} */ @Deprecated @Experimental(Kind.CODER_CONTEXT) @@ -152,6 +160,8 @@ public abstract class Coder<T> implements Serializable { * @throws IOException if reading from the {@code InputStream} fails * for some reason * @throws CoderException if the value could not be decoded for some reason + * + * @deprecated only implement and call {@link #decode(InputStream)} */ @Deprecated @Experimental(Kind.CODER_CONTEXT) http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java index 2ba548a..53cb6d3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java @@ -234,6 +234,9 @@ public class CoderRegistry { * type uses the given {@link Coder}. * * @throws CannotProvideCoderException if a {@link Coder} cannot be provided + * + * @deprecated This method is to change in an unknown backwards incompatible way once support for + * this functionality is refined. */ @Deprecated @Internal @@ -254,6 +257,9 @@ public class CoderRegistry { * used for its input elements. * * @throws CannotProvideCoderException if a {@link Coder} cannot be provided + * + * @deprecated This method is to change in an unknown backwards incompatible way once support for + * this functionality is refined. */ @Deprecated @Internal @@ -276,6 +282,9 @@ public class CoderRegistry { * subclass, given {@link Coder Coders} to use for all other type parameters (if any). * * @throws CannotProvideCoderException if a {@link Coder} cannot be provided + * + * @deprecated This method is to change in an unknown backwards incompatible way once support for + * this functionality is refined. */ @Deprecated @Internal http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java index 37bbe46..575218b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java @@ -237,12 +237,6 @@ public class AvroSource<T> extends BlockBasedSource<T> { super.validate(); } - @Deprecated // Added to let DataflowRunner migrate off of this; to be deleted. - public BlockBasedSource<T> createForSubrangeOfFile(String fileName, long start, long end) - throws IOException { - return createForSubrangeOfFile(FileSystems.matchSingleFileSpec(fileName), start, end); - } - @Override public BlockBasedSource<T> createForSubrangeOfFile(Metadata fileMetadata, long start, long end) { byte[] syncMarker = this.syncMarker; http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java index 6e2b8c6..ed80f2f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java @@ -861,8 +861,9 @@ public class PAssert { } /** - * Always throws an {@link UnsupportedOperationException}: users are probably looking for - * {@link #isEqualTo}. + * @throws UnsupportedOperationException always + * @deprecated {@link Object#equals(Object)} is not supported on PAssert objects. If you meant + * to test object equality, use {@link #isEqualTo} instead. */ @Deprecated @Override http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StreamingIT.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StreamingIT.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StreamingIT.java index 427b908..475372d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StreamingIT.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StreamingIT.java @@ -29,6 +29,10 @@ package org.apache.beam.sdk.testing; * StreamingPipeline.main(...); * } * </code></pre> + * + * @deprecated tests which use unbounded PCollections should be in the category + * {@link UsesUnboundedPCollections}. Beyond that, it is up to the runner and test configuration + * to decide whether to run in streaming mode. */ @Deprecated public interface StreamingIT { http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index d7effb5..c195352 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -1416,7 +1416,6 @@ public class Combine { * Returns a {@code CombineFn} that uses the given * {@code SerializableFunction} to combine values. */ - @Deprecated public static <V> SimpleCombineFn<V> of( SerializableFunction<Iterable<V>, V> combiner) { return new SimpleCombineFn<>(combiner); http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index 1b809c2..37c6263 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -728,6 +728,9 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD /** * Finalize the {@link DoFn} construction to prepare for processing. * This method should be called by runners before any processing methods. + * + * @deprecated use {@link Setup} or {@link StartBundle} instead. This method will be removed in a + * future release. */ @Deprecated public final void prepareForProcessing() {} http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/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 331b143..c94fad6 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 @@ -497,7 +497,7 @@ public class 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 + * @deprecated 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. http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java index 33c5a6a..44b87a0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.transforms.reflect; -import java.io.Serializable; import org.apache.beam.sdk.transforms.DoFn; /** Static utilities for working with {@link DoFnInvoker}. */ @@ -36,13 +35,5 @@ public class DoFnInvokers { return ByteBuddyDoFnInvokerFactory.only().newByteBuddyInvoker(fn); } - /** TODO: remove this when Dataflow worker uses the DoFn overload. */ - @Deprecated - @SuppressWarnings({"unchecked"}) - public static <InputT, OutputT> DoFnInvoker<InputT, OutputT> invokerFor( - Serializable fn) { - return invokerFor((DoFn) fn); - } - private DoFnInvokers() {} } http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java index ef6d833..54ac77c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java @@ -111,7 +111,6 @@ public class IdentityWindowFn<T> extends NonMergingWindowFn<T, BoundedWindow> { getClass().getCanonicalName())); } - @Deprecated @Override public Instant getOutputTime(Instant inputTimestamp, BoundedWindow window) { return inputTimestamp; http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java index e17e146..f2a3097 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java @@ -186,7 +186,6 @@ public class PCollectionViews { /** * Returns if a default value was specified. */ - @Deprecated @Internal public boolean hasDefault() { return hasDefault; http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java ---------------------------------------------------------------------- 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 1089028..71f9465 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 @@ -37,8 +37,8 @@ public interface PValue extends POutput, PInput { /** * {@inheritDoc}. * - * <p>A {@link PValue} always expands into itself. Calling {@link #expand()} on a PValue is almost - * never appropriate. + * @deprecated A {@link PValue} always expands into itself. Calling {@link #expand()} on a PValue + * is almost never appropriate. */ @Deprecated Map<TupleTag<?>, PValue> expand(); http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java index aa8d94c..274fef4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java @@ -89,7 +89,8 @@ public class DefaultCoderTest { private static class OldCustomSerializableCoder extends SerializableCoder<OldCustomRecord> { // Extending SerializableCoder isn't trivial, but it can be done. - @Deprecated // old form using a Class + + // Old form using a Class. @SuppressWarnings("unchecked") public static <T extends Serializable> SerializableCoder<T> of(Class<T> recordType) { checkArgument(OldCustomRecord.class.isAssignableFrom(recordType)); http://git-wip-us.apache.org/repos/asf/beam/blob/d2901145/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java ---------------------------------------------------------------------- diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java index 977e803..4702e05 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java @@ -114,9 +114,9 @@ public class BoundedSourceRunner<InputT extends BoundedSource<OutputT>, OutputT> } /** - * The runner harness is meant to send the source over the Beam Fn Data API which would be - * consumed by the {@link #runReadLoop}. Drop this method once the runner harness sends the - * source instead of unpacking it from the data block of the function specification. + * @deprecated The runner harness is meant to send the source over the Beam Fn Data API which + * would be consumed by the {@link #runReadLoop}. Drop this method once the runner harness sends + * the source instead of unpacking it from the data block of the function specification. */ @Deprecated public void start() throws Exception {
