Delete long-deprecated StateContexts methods
Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/b12e5ffb Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/b12e5ffb Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/b12e5ffb Branch: refs/heads/master Commit: b12e5ffbaf9bacebc5d80b3ed24c37a22e37524f Parents: af391b8 Author: Kenneth Knowles <k...@google.com> Authored: Thu Jan 26 21:05:09 2017 -0800 Committer: Kenneth Knowles <k...@google.com> Committed: Mon Feb 6 09:26:06 2017 -0800 ---------------------------------------------------------------------- .../runners/core/SimpleOldDoFnRunnerTest.java | 1 - .../core/triggers/NeverStateMachineTest.java | 1 - .../beam/sdk/util/state/StateContexts.java | 63 -------------------- 3 files changed, 65 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/b12e5ffb/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java ---------------------------------------------------------------------- diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java index 97da9ee..28698ca 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java @@ -23,7 +23,6 @@ import static org.mockito.Mockito.mock; import java.util.Arrays; import java.util.List; import org.apache.beam.runners.core.BaseExecutionContext.StepContext; - import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; http://git-wip-us.apache.org/repos/asf/beam/blob/b12e5ffb/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java ---------------------------------------------------------------------- diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java index 6d8a344..79147d4 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java @@ -25,7 +25,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.values.TimestampedValue; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; http://git-wip-us.apache.org/repos/asf/beam/blob/b12e5ffb/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java index a9ce0b5..2ce9594 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java @@ -17,10 +17,8 @@ */ package org.apache.beam.sdk.util.state; -import javax.annotation.Nullable; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.values.PCollectionView; /** @@ -50,65 +48,4 @@ public class StateContexts { public static <W extends BoundedWindow> StateContext<W> nullContext() { return (StateContext<W>) NULL_CONTEXT; } - - /** - * @deprecated This exists only for temporary compatibility with Dataflow worker and should be - * deleted once a worker image is released that uses runners-core build after - * https://github.com/apache/beam/pull/1353. - */ - @Deprecated - public static <W extends BoundedWindow> StateContext<W> windowOnly(final W window) { - return new StateContext<W>() { - @Override - public PipelineOptions getPipelineOptions() { - throw new IllegalArgumentException( - "cannot call getPipelineOptions() in a window only context"); - } - - @Override - public <T> T sideInput(PCollectionView<T> view) { - throw new IllegalArgumentException("cannot call sideInput() in a window only context"); - } - - @Override - public W window() { - return window; - } - }; - } - - /** - * @deprecated This exists only for temporary compatibility with Dataflow worker and should be - * deleted once a worker image is released that uses runners-core build after - * https://github.com/apache/beam/pull/1353. - */ - @Deprecated - public static <W extends BoundedWindow> StateContext<W> createFromComponents( - @Nullable final PipelineOptions options, - final WindowingInternals<?, ?> windowingInternals, - final W window) { - @SuppressWarnings("unchecked") - StateContext<W> typedNullContext = (StateContext<W>) NULL_CONTEXT; - if (options == null) { - return typedNullContext; - } else { - return new StateContext<W>() { - - @Override - public PipelineOptions getPipelineOptions() { - return options; - } - - @Override - public <T> T sideInput(PCollectionView<T> view) { - return windowingInternals.sideInput(view, window); - } - - @Override - public W window() { - return window; - } - }; - } - } }