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/05/05 20:16:51 UTC

[3/4] beam git commit: Move PipelineRunner to toplevel sdk package (automated refactor)

Move PipelineRunner to toplevel sdk package (automated refactor)

This allows excluding the runner-author-only sdk.runners package from the
public javadoc.


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

Branch: refs/heads/master
Commit: c640e7437871313303636def48d63acd678a3430
Parents: a1425e1
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri May 5 10:36:40 2017 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri May 5 13:15:25 2017 -0700

----------------------------------------------------------------------
 .../apache/beam/runners/apex/ApexRunner.java    |  2 +-
 .../beam/runners/apex/ApexRunnerRegistrar.java  |  2 +-
 .../beam/runners/apex/TestApexRunner.java       |  2 +-
 .../org/apache/beam/runners/core/OldDoFn.java   |  4 +-
 .../beam/runners/direct/DirectRegistrar.java    |  2 +-
 .../beam/runners/direct/DirectRunner.java       |  2 +-
 .../runners/direct/DoFnLifecycleManager.java    |  2 +-
 .../beam/runners/direct/DirectRunnerTest.java   |  2 +-
 .../apache/beam/runners/flink/FlinkRunner.java  |  2 +-
 .../runners/flink/FlinkRunnerRegistrar.java     |  2 +-
 .../beam/runners/flink/TestFlinkRunner.java     |  2 +-
 .../dataflow/DataflowPipelineRegistrar.java     |  2 +-
 .../beam/runners/dataflow/DataflowRunner.java   |  2 +-
 .../dataflow/testing/TestDataflowRunner.java    |  2 +-
 .../apache/beam/runners/spark/SparkRunner.java  |  2 +-
 .../beam/runners/spark/SparkRunnerDebugger.java |  2 +-
 .../runners/spark/SparkRunnerRegistrar.java     |  2 +-
 .../beam/runners/spark/TestSparkRunner.java     |  2 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java |  1 -
 .../org/apache/beam/sdk/PipelineRunner.java     | 61 +++++++++++++++++++
 .../java/org/apache/beam/sdk/coders/Coder.java  |  9 +--
 .../apache/beam/sdk/coders/package-info.java    |  2 +-
 .../java/org/apache/beam/sdk/io/AvroSource.java |  2 +-
 .../beam/sdk/options/PipelineOptions.java       |  2 +-
 .../sdk/options/PipelineOptionsFactory.java     |  2 +-
 .../java/org/apache/beam/sdk/package-info.java  |  2 +-
 .../sdk/runners/PTransformOverrideFactory.java  |  1 +
 .../apache/beam/sdk/runners/PipelineRunner.java | 63 --------------------
 .../sdk/runners/PipelineRunnerRegistrar.java    |  1 +
 .../org/apache/beam/sdk/state/StateContext.java |  3 +-
 .../apache/beam/sdk/testing/CrashingRunner.java |  2 +-
 .../apache/beam/sdk/testing/NeedsRunner.java    |  2 +-
 .../org/apache/beam/sdk/testing/PAssert.java    |  2 +-
 .../org/apache/beam/sdk/testing/TestStream.java |  2 +-
 .../beam/sdk/transforms/CombineWithContext.java |  3 +-
 .../org/apache/beam/sdk/transforms/DoFn.java    |  6 +-
 .../beam/sdk/transforms/Materialization.java    |  2 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   |  2 +-
 .../org/apache/beam/sdk/transforms/View.java    |  2 +-
 .../apache/beam/sdk/values/PCollectionView.java |  3 +-
 .../java/org/apache/beam/sdk/PipelineTest.java  |  1 -
 .../sdk/options/PipelineOptionsFactoryTest.java |  2 +-
 .../beam/sdk/runners/PipelineRunnerTest.java    |  1 +
 .../beam/sdk/testing/CrashingRunnerTest.java    |  2 +-
 .../display/DisplayDataEvaluator.java           |  2 +-
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    |  2 +-
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |  2 +-
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  |  3 +-
 .../apache/beam/sdk/io/gcp/pubsub/PubsubIO.java |  2 +-
 .../java/org/apache/beam/sdk/io/xml/XmlIO.java  |  4 +-
 50 files changed, 119 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java
----------------------------------------------------------------------
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java
index ede41bc..a50e10e 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java
@@ -43,6 +43,7 @@ import org.apache.beam.runners.core.construction.PTransformReplacements;
 import org.apache.beam.runners.core.construction.PrimitiveCreate;
 import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderRegistry;
 import org.apache.beam.sdk.coders.ListCoder;
@@ -50,7 +51,6 @@ import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsValidator;
 import org.apache.beam.sdk.runners.AppliedPTransform;
 import org.apache.beam.sdk.runners.PTransformOverride;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.Combine.GloballyAsSingletonView;
 import org.apache.beam.sdk.transforms.Create;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerRegistrar.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerRegistrar.java
index 8cde692..2aa34ef 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerRegistrar.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerRegistrar.java
@@ -20,9 +20,9 @@ package org.apache.beam.runners.apex;
 
 import com.google.auto.service.AutoService;
 import com.google.common.collect.ImmutableList;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
 
 /**

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java
----------------------------------------------------------------------
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java
index a64ac54..e068db0 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java
@@ -19,9 +19,9 @@ package org.apache.beam.runners.apex;
 
 import java.io.IOException;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsValidator;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.joda.time.Duration;
 
 /**

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/core-java/src/main/java/org/apache/beam/runners/core/OldDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/OldDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/OldDoFn.java
index 419c837..41bb598 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OldDoFn.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OldDoFn.java
@@ -18,9 +18,9 @@
 package org.apache.beam.runners.core;
 
 import java.io.Serializable;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFnTester;
 import org.apache.beam.sdk.transforms.ParDo;
@@ -71,7 +71,7 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
 
     /**
      * Returns the {@code PipelineOptions} specified with the
-     * {@link org.apache.beam.sdk.runners.PipelineRunner}
+     * {@link PipelineRunner}
      * invoking this {@code OldDoFn}.  The {@code PipelineOptions} will
      * be the default running via {@link DoFnTester}.
      */

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java
index 9a34d3d..3e69e2b 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java
@@ -19,9 +19,9 @@ package org.apache.beam.runners.direct;
 
 import com.google.auto.service.AutoService;
 import com.google.common.collect.ImmutableList;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
 
 /**

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index 984598a..b0ce5eb 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -34,12 +34,12 @@ import org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStrea
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
 import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.sdk.metrics.MetricsEnvironment;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.runners.PTransformOverride;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.testing.TestStream;
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.PTransform;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
index cd644a6..0e30e5c 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
@@ -26,7 +26,7 @@ import com.google.common.cache.RemovalNotification;
 import java.util.Collection;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFn.Setup;
 import org.apache.beam.sdk.transforms.DoFn.Teardown;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
index 83881fc..428c6fc 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
@@ -43,6 +43,7 @@ import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.coders.CustomCoder;
@@ -55,7 +56,6 @@ import org.apache.beam.sdk.io.GenerateSequence;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Count;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
index a5972ef..80ef7bb 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
@@ -31,10 +31,10 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.metrics.MetricsEnvironment;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsValidator;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.View;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
index 681459a..ebc6adc 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
@@ -20,9 +20,9 @@ package org.apache.beam.runners.flink;
 
 import com.google.auto.service.AutoService;
 import com.google.common.collect.ImmutableList;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
 
 

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java
index 8f50105..01b67e5 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java
@@ -20,10 +20,10 @@ package org.apache.beam.runners.flink;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
 import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.options.PipelineOptionsValidator;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.util.UserCodeException;
 
 /**

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java
index 5bd3bcd..b6802bb 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java
@@ -20,9 +20,9 @@ package org.apache.beam.runners.dataflow;
 import com.google.auto.service.AutoService;
 import com.google.common.collect.ImmutableList;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
 
 /**

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/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 4198bae..5278a4a 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
@@ -77,6 +77,7 @@ import org.apache.beam.runners.dataflow.util.PropertyNames;
 import org.apache.beam.sdk.Pipeline;
 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.coders.Coder;
 import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
@@ -102,7 +103,6 @@ import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider;
 import org.apache.beam.sdk.runners.AppliedPTransform;
 import org.apache.beam.sdk.runners.PTransformOverride;
 import org.apache.beam.sdk.runners.PTransformOverrideFactory;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.transforms.Combine;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java
index c238d80..ce91915 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java
@@ -38,8 +38,8 @@ import org.apache.beam.runners.dataflow.util.MonitoringUtil;
 import org.apache.beam.runners.dataflow.util.MonitoringUtil.JobMessagesHandler;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.testing.TestPipelineOptions;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
index 16d9fdc..8c02f0f 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
@@ -41,13 +41,13 @@ import org.apache.beam.runners.spark.translation.streaming.Checkpoint.Checkpoint
 import org.apache.beam.runners.spark.translation.streaming.SparkRunnerStreamingContextFactory;
 import org.apache.beam.runners.spark.util.GlobalWatermarkHolder.WatermarksListener;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.metrics.MetricsEnvironment;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.options.PipelineOptionsValidator;
 import org.apache.beam.sdk.runners.AppliedPTransform;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.PTransform;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java
index 11c52c7..6c74146 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java
@@ -25,9 +25,9 @@ import org.apache.beam.runners.spark.translation.SparkPipelineTranslator;
 import org.apache.beam.runners.spark.translation.TransformTranslator;
 import org.apache.beam.runners.spark.translation.streaming.StreamingTransformTranslator;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsValidator;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.streaming.api.java.JavaStreamingContext;
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java
index bf926dc..e2e5ceb 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java
@@ -20,9 +20,9 @@ package org.apache.beam.runners.spark;
 
 import com.google.auto.service.AutoService;
 import com.google.common.collect.ImmutableList;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
 
 /**

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java
index ada7fb5..6d10b75 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java
@@ -39,13 +39,13 @@ import org.apache.beam.runners.spark.stateful.SparkTimerInternals;
 import org.apache.beam.runners.spark.util.GlobalWatermarkHolder;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.io.BoundedReadFromUnboundedSource;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsValidator;
 import org.apache.beam.sdk.runners.AppliedPTransform;
 import org.apache.beam.sdk.runners.PTransformOverride;
 import org.apache.beam.sdk.runners.PTransformOverrideFactory;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.PBegin;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
index 1769c05..f4da6ad 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
@@ -38,7 +38,6 @@ import org.apache.beam.sdk.runners.PTransformOverride;
 import org.apache.beam.sdk.runners.PTransformOverrideFactory;
 import org.apache.beam.sdk.runners.PTransformOverrideFactory.PTransformReplacement;
 import org.apache.beam.sdk.runners.PTransformOverrideFactory.ReplacementOutput;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.transforms.Create;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineRunner.java
new file mode 100644
index 0000000..18e79eb
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineRunner.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.util.InstanceBuilder;
+
+/**
+ * A {@link PipelineRunner} can execute, translate, or otherwise process a
+ * {@link Pipeline}.
+ *
+ * @param <ResultT> the type of the result of {@link #run}.
+ */
+public abstract class PipelineRunner<ResultT extends PipelineResult> {
+
+  /**
+   * Constructs a runner from the provided options.
+   *
+   * @return The newly created runner.
+   */
+  public static PipelineRunner<? extends PipelineResult> fromOptions(PipelineOptions options) {
+    checkNotNull(options);
+    PipelineOptionsValidator.validate(PipelineOptions.class, options);
+
+    // (Re-)register standard FileSystems. Clobbers any prior credentials.
+    FileSystems.setDefaultConfigInWorkers(options);
+
+    @SuppressWarnings("unchecked")
+    PipelineRunner<? extends PipelineResult> result =
+        InstanceBuilder.ofType(PipelineRunner.class)
+        .fromClass(options.getRunner())
+        .fromFactoryMethod("fromOptions")
+        .withArg(PipelineOptions.class, options)
+        .build();
+    return result;
+  }
+
+  /**
+   * Processes the given Pipeline, returning the results.
+   */
+  public abstract ResultT run(Pipeline pipeline);
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/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 169e448..061e9e5 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
@@ -29,6 +29,7 @@ import java.io.Serializable;
 import java.util.Arrays;
 import java.util.List;
 import javax.annotation.Nullable;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
@@ -243,7 +244,7 @@ public abstract class Coder<T> implements Serializable {
    * constant time (or lazily).
    *
    * <p>Not intended to be called by user code, but instead by
-   * {@link org.apache.beam.sdk.runners.PipelineRunner}
+   * {@link PipelineRunner}
    * implementations.
    */
   public abstract boolean isRegisterByteSizeObserverCheap(T value);
@@ -255,7 +256,7 @@ public abstract class Coder<T> implements Serializable {
    * constant time (or lazily).
    *
    * <p>Not intended to be called by user code, but instead by
-   * {@link org.apache.beam.sdk.runners.PipelineRunner}
+   * {@link PipelineRunner}
    * implementations.
    */
   @Deprecated
@@ -266,7 +267,7 @@ public abstract class Coder<T> implements Serializable {
    * of the encoded value using this {@code Coder}.
    *
    * <p>Not intended to be called by user code, but instead by
-   * {@link org.apache.beam.sdk.runners.PipelineRunner}
+   * {@link PipelineRunner}
    * implementations.
    */
   public abstract void registerByteSizeObserver(
@@ -278,7 +279,7 @@ public abstract class Coder<T> implements Serializable {
    * of the encoded value using this {@code Coder}.
    *
    * <p>Not intended to be called by user code, but instead by
-   * {@link org.apache.beam.sdk.runners.PipelineRunner}
+   * {@link PipelineRunner}
    * implementations.
    */
   @Deprecated

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/package-info.java
index 84e59e0..5693077 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/package-info.java
@@ -27,7 +27,7 @@
  * machines.
  *
  * <p>Exactly when PCollection elements are encoded during execution depends on which
- * {@link org.apache.beam.sdk.runners.PipelineRunner} is being used and how that runner
+ * {@link org.apache.beam.sdk.PipelineRunner} is being used and how that runner
  * chooses to execute the pipeline. As such, Beam requires that all PCollections have an
  * appropriate Coder in case it becomes necessary. In many cases, the Coder can be inferred from
  * the available Java type

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/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 96d21c6..61bc4a4 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
@@ -48,12 +48,12 @@ import org.apache.avro.io.DatumReader;
 import org.apache.avro.io.DecoderFactory;
 import org.apache.avro.reflect.ReflectData;
 import org.apache.avro.reflect.ReflectDatumReader;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.io.fs.MatchResult.Metadata;
 import org.apache.beam.sdk.io.fs.ResourceId;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
 import org.apache.commons.compress.compressors.snappy.SnappyCompressorInputStream;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
index 37081fd..9a4d25a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
@@ -30,9 +30,9 @@ import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicLong;
 import javax.annotation.concurrent.ThreadSafe;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.ProxyInvocationHandler.Deserializer;
 import org.apache.beam.sdk.options.ProxyInvocationHandler.Serializer;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.display.HasDisplayData;
 import org.joda.time.DateTimeUtils;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
index 0f6bedf..c0990cb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
@@ -73,8 +73,8 @@ import java.util.SortedSet;
 import java.util.TreeMap;
 import java.util.TreeSet;
 import javax.annotation.Nonnull;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.Validation.Required;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.util.StringUtils;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/package-info.java
index 166c276..995bcb9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/package-info.java
@@ -27,7 +27,7 @@
  * the PTransforms consume and produce.
  *
  * <p>Each Pipeline has a
- * {@link org.apache.beam.sdk.runners.PipelineRunner} to specify
+ * {@link org.apache.beam.sdk.PipelineRunner} to specify
  * where and how it should run after pipeline construction is complete.
  *
  */

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java
index 5f012de..58314c4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java
@@ -21,6 +21,7 @@ package org.apache.beam.sdk.runners;
 
 import com.google.auto.value.AutoValue;
 import java.util.Map;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 import org.apache.beam.sdk.annotations.Internal;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
deleted file mode 100644
index 229e04f..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.io.FileSystems;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsValidator;
-import org.apache.beam.sdk.util.InstanceBuilder;
-
-/**
- * A {@link PipelineRunner} can execute, translate, or otherwise process a
- * {@link Pipeline}.
- *
- * @param <ResultT> the type of the result of {@link #run}.
- */
-public abstract class PipelineRunner<ResultT extends PipelineResult> {
-
-  /**
-   * Constructs a runner from the provided options.
-   *
-   * @return The newly created runner.
-   */
-  public static PipelineRunner<? extends PipelineResult> fromOptions(PipelineOptions options) {
-    checkNotNull(options);
-    PipelineOptionsValidator.validate(PipelineOptions.class, options);
-
-    // (Re-)register standard FileSystems. Clobbers any prior credentials.
-    FileSystems.setDefaultConfigInWorkers(options);
-
-    @SuppressWarnings("unchecked")
-    PipelineRunner<? extends PipelineResult> result =
-        InstanceBuilder.ofType(PipelineRunner.class)
-        .fromClass(options.getRunner())
-        .fromFactoryMethod("fromOptions")
-        .withArg(PipelineOptions.class, options)
-        .build();
-    return result;
-  }
-
-  /**
-   * Processes the given Pipeline, returning the results.
-   */
-  public abstract ResultT run(Pipeline pipeline);
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java
index 41fd6f0..c6303dd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java
@@ -19,6 +19,7 @@ package org.apache.beam.sdk.runners;
 
 import com.google.auto.service.AutoService;
 import java.util.ServiceLoader;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.annotations.Internal;
 
 /**

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java
index 52177cc..b7dd8e2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.state;
 
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.annotations.Internal;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -30,7 +31,7 @@ import org.apache.beam.sdk.values.PCollectionView;
 public interface StateContext<W extends BoundedWindow> {
   /**
    * Returns the {@code PipelineOptions} specified with the
-   * {@link org.apache.beam.sdk.runners.PipelineRunner}.
+   * {@link PipelineRunner}.
    */
   PipelineOptions getPipelineOptions();
 

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CrashingRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CrashingRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CrashingRunner.java
index b3360ae..0dea274 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CrashingRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CrashingRunner.java
@@ -20,8 +20,8 @@ package org.apache.beam.sdk.testing;
 
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.PipelineRunner;
 
 /**
  * A {@link PipelineRunner} that applies no overrides and throws an exception on calls to

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/NeedsRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/NeedsRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/NeedsRunner.java
index 5f22bee..8ebc76c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/NeedsRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/NeedsRunner.java
@@ -18,7 +18,7 @@
 
 package org.apache.beam.sdk.testing;
 
-import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.PipelineRunner;
 
 /**
  * Category tag for validation tests which utilize {@link TestPipeline} for execution and expect

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/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 b5d7db5..6e2b8c6 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
@@ -34,6 +34,7 @@ import java.util.Map;
 import java.util.NoSuchElementException;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.coders.IterableCoder;
@@ -42,7 +43,6 @@ import org.apache.beam.sdk.coders.MapCoder;
 import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.metrics.Counter;
 import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java
index 446c8a5..d41b9ef 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java
@@ -25,8 +25,8 @@ import com.google.auto.value.AutoValue;
 import com.google.common.collect.ImmutableList;
 import java.util.List;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.values.PBegin;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java
index 9ae19f8..a7601b3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.transforms;
 
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.values.PCollectionView;
@@ -36,7 +37,7 @@ public class CombineWithContext {
   public abstract static class Context {
     /**
      * Returns the {@code PipelineOptions} specified with the
-     * {@link org.apache.beam.sdk.runners.PipelineRunner}
+     * {@link PipelineRunner}
      * invoking this {@code KeyedCombineFn}.
      */
     public abstract PipelineOptions getPipelineOptions();

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/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 befba1d..e711ac2 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
@@ -88,7 +88,7 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
   public abstract class StartBundleContext {
     /**
      * Returns the {@code PipelineOptions} specified with the {@link
-     * org.apache.beam.sdk.runners.PipelineRunner} invoking this {@code DoFn}. The {@code
+     * org.apache.beam.sdk.PipelineRunner} invoking this {@code DoFn}. The {@code
      * PipelineOptions} will be the default running via {@link DoFnTester}.
      */
     public abstract PipelineOptions getPipelineOptions();
@@ -100,7 +100,7 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
   public abstract class FinishBundleContext {
     /**
      * Returns the {@code PipelineOptions} specified with the {@link
-     * org.apache.beam.sdk.runners.PipelineRunner} invoking this {@code DoFn}. The {@code
+     * org.apache.beam.sdk.PipelineRunner} invoking this {@code DoFn}. The {@code
      * PipelineOptions} will be the default running via {@link DoFnTester}.
      */
     public abstract PipelineOptions getPipelineOptions();
@@ -136,7 +136,7 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
   public abstract class WindowedContext {
     /**
      * Returns the {@code PipelineOptions} specified with the {@link
-     * org.apache.beam.sdk.runners.PipelineRunner} invoking this {@code DoFn}. The {@code
+     * org.apache.beam.sdk.PipelineRunner} invoking this {@code DoFn}. The {@code
      * PipelineOptions} will be the default running via {@link DoFnTester}.
      */
     public abstract PipelineOptions getPipelineOptions();

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java
index 6fb8c29..0d02b32 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java
@@ -18,8 +18,8 @@
 
 package org.apache.beam.sdk.transforms;
 
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.annotations.Internal;
-import org.apache.beam.sdk.runners.PipelineRunner;
 
 /**
  * <b><i>For internal use only; no backwards-compatibility guarantees.</i></b>

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/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 c45311a..e67dbe1 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
@@ -29,10 +29,10 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.state.StateSpec;
 import org.apache.beam.sdk.transforms.DoFn.WindowedContext;
 import org.apache.beam.sdk.transforms.display.DisplayData;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/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 d7b8145..bcbdb24 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
@@ -19,10 +19,10 @@ package org.apache.beam.sdk.transforms;
 
 import java.util.List;
 import java.util.Map;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.annotations.Internal;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.values.KV;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java
index f89041a..7d87412 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java
@@ -19,6 +19,7 @@ package org.apache.beam.sdk.values;
 
 import java.io.Serializable;
 import javax.annotation.Nullable;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 import org.apache.beam.sdk.annotations.Internal;
@@ -36,7 +37,7 @@ import org.apache.beam.sdk.util.WindowedValue;
  *
  * <p>A {@link PCollectionView} should always be the output of a
  * {@link org.apache.beam.sdk.transforms.PTransform}. It is the joint responsibility of
- * this transform and each {@link org.apache.beam.sdk.runners.PipelineRunner} to implement
+ * this transform and each {@link PipelineRunner} to implement
  * the view in a runner-specific manner.
  *
  * <p>The most common case is using the {@link View} transforms to prepare a {@link PCollection}

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
index d6b527c..2cc3f04 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
@@ -44,7 +44,6 @@ import org.apache.beam.sdk.runners.AppliedPTransform;
 import org.apache.beam.sdk.runners.PTransformMatcher;
 import org.apache.beam.sdk.runners.PTransformOverride;
 import org.apache.beam.sdk.runners.PTransformOverrideFactory;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.testing.CrashingRunner;
 import org.apache.beam.sdk.testing.ExpectedLogs;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
index 76a5f18..d40b5fc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
@@ -53,7 +53,7 @@ import java.util.Map;
 import java.util.Set;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
 import org.apache.beam.sdk.testing.CrashingRunner;
 import org.apache.beam.sdk.testing.ExpectedLogs;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
index 76d8627..c5d7fbf 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
@@ -19,6 +19,7 @@ package org.apache.beam.sdk.runners;
 
 import static org.junit.Assert.assertTrue;
 
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.CrashingRunner;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java
index c66aa50..62c5134 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java
@@ -22,9 +22,9 @@ import static org.junit.Assert.assertTrue;
 
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.Create;
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
index 8f57f45..6e8ca5e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
@@ -21,10 +21,10 @@ import com.google.common.collect.Sets;
 import java.util.Objects;
 import java.util.Set;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
index 6e410c7..304864a 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
@@ -43,6 +43,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.regex.Pattern;
 import javax.annotation.Nullable;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.KvCoder;
@@ -66,7 +67,6 @@ import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.ValueProvider;
 import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider;
 import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.transforms.display.DisplayData;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
index 0e97c12..846d07b 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
@@ -42,6 +42,7 @@ import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import javax.annotation.Nullable;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
@@ -52,7 +53,6 @@ import org.apache.beam.sdk.io.range.ByteKey;
 import org.apache.beam.sdk.io.range.ByteKeyRange;
 import org.apache.beam.sdk.io.range.ByteKeyRangeTracker;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
index fd4fccf..d6464dd 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
@@ -69,6 +69,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.NoSuchElementException;
 import javax.annotation.Nullable;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 import org.apache.beam.sdk.coders.SerializableCoder;
@@ -192,7 +193,7 @@ import org.slf4j.LoggerFactory;
  * by providing the host port information through {@code withLocalhost("host:port"} for all the
  * above transforms. In such a case, all the Cloud Datastore API calls are directed to the Emulator.
  *
- * @see org.apache.beam.sdk.runners.PipelineRunner
+ * @see PipelineRunner
  */
 public class DatastoreV1 {
 

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java
index 51da111..ac6cb44 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java
@@ -31,6 +31,7 @@ import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import javax.annotation.Nullable;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
@@ -44,7 +45,6 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath;
 import org.apache.beam.sdk.options.ValueProvider;
 import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider;
 import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.MapElements;
 import org.apache.beam.sdk.transforms.PTransform;

http://git-wip-us.apache.org/repos/asf/beam/blob/c640e743/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java
index a34fb0f..7255a94 100644
--- a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java
+++ b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java
@@ -26,6 +26,7 @@ import javax.annotation.Nullable;
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
 import javax.xml.bind.ValidationEventHandler;
+import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.io.BoundedSource;
 import org.apache.beam.sdk.io.CompressedSource;
 import org.apache.beam.sdk.io.FileBasedSink;
@@ -34,7 +35,6 @@ import org.apache.beam.sdk.io.fs.ResourceId;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.ValueProvider;
 import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.values.PBegin;
@@ -102,7 +102,7 @@ public class XmlIO {
    *
    * <h3>Permissions</h3>
    *
-   * <p>Permission requirements depend on the {@link org.apache.beam.sdk.runners.PipelineRunner
+   * <p>Permission requirements depend on the {@link PipelineRunner
    * PipelineRunner} that is used to execute the Beam pipeline. Please refer to the documentation of
    * corresponding {@link PipelineRunner PipelineRunners} for more details.
    *