You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by dh...@apache.org on 2016/10/12 18:52:32 UTC

incubator-beam git commit: Minor rewording to drop Dataflow from Beam SDK core

Repository: incubator-beam
Updated Branches:
  refs/heads/master 4d10f86cf -> c561cd9c1


Minor rewording to drop Dataflow from Beam SDK core


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

Branch: refs/heads/master
Commit: c561cd9c1be6e77568b129792ace3d8d52bc329b
Parents: 4d10f86
Author: Dan Halperin <dh...@google.com>
Authored: Mon Sep 26 22:43:40 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Oct 12 11:49:37 2016 -0700

----------------------------------------------------------------------
 examples/java/pom.xml                             |  2 +-
 pom.xml                                           |  4 ++--
 runners/google-cloud-dataflow-java/pom.xml        |  2 +-
 sdks/java/core/pom.xml                            |  2 +-
 .../java/org/apache/beam/sdk/coders/Coder.java    |  2 +-
 .../org/apache/beam/sdk/coders/package-info.java  |  2 +-
 .../java/org/apache/beam/sdk/io/PubsubIO.java     | 18 +++++++++---------
 .../main/java/org/apache/beam/sdk/io/Sink.java    |  2 +-
 .../main/java/org/apache/beam/sdk/io/TextIO.java  | 15 ++-------------
 .../java/org/apache/beam/sdk/io/XmlSource.java    | 13 +++++++------
 .../apache/beam/sdk/io/range/ByteKeyRange.java    |  6 +++---
 .../java/org/apache/beam/sdk/package-info.java    |  2 +-
 .../org/apache/beam/sdk/runners/package-info.java | 17 +++--------------
 .../org/apache/beam/sdk/testing/package-info.java |  4 ++--
 .../org/apache/beam/sdk/transforms/OldDoFn.java   | 15 ++++++++-------
 .../apache/beam/sdk/transforms/PTransform.java    |  2 +-
 .../org/apache/beam/sdk/transforms/ParDo.java     |  7 +++----
 .../org/apache/beam/sdk/util/ReleaseInfo.java     |  6 +++---
 .../org/apache/beam/sdk/values/package-info.java  |  4 ++--
 19 files changed, 52 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/examples/java/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java/pom.xml b/examples/java/pom.xml
index aa7c22a..d18f959 100644
--- a/examples/java/pom.xml
+++ b/examples/java/pom.xml
@@ -296,7 +296,7 @@
           <doctitle>Apache Beam Examples</doctitle>
 
           <subpackages>org.apache.beam.examples</subpackages>
-          <additionalparam>-exclude org.apache.beam.sdk.runners.worker:org.apache.beam.sdk.runners.dataflow:org.apache.beam.sdk.util ${dataflow.javadoc_opts}</additionalparam>
+          <additionalparam>-exclude org.apache.beam.sdk.runners.worker:org.apache.beam.sdk.runners.dataflow:org.apache.beam.sdk.util ${beam.javadoc_opts}</additionalparam>
           <use>false</use>
           <quiet>true</quiet>
           <bottom><![CDATA[<br>]]></bottom>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c135f17..8a28bd9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -92,7 +92,7 @@
 
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <dataflow.javadoc_opts/>
+    <beam.javadoc_opts/>
 
     <!-- Disable integration tests by default -->
     <skipITs>true</skipITs>
@@ -224,7 +224,7 @@
         <jdk>[1.8,)</jdk>
       </activation>
       <properties>
-        <dataflow.javadoc_opts>-Xdoclint:-missing</dataflow.javadoc_opts>
+        <beam.javadoc_opts>-Xdoclint:-missing</beam.javadoc_opts>
       </properties>
     </profile>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/runners/google-cloud-dataflow-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml
index bf4d93d..b035028 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -98,7 +98,7 @@
           <doctitle>Google Cloud Dataflow Runner for Java, version ${project.version}</doctitle>
 
           <subpackages>org.apache.beam.runners.dataflow</subpackages>
-          <additionalparam>-exclude org.apache.beam.sdk.runners.dataflow.internal:org.apache.beam.sdk.runners.dataflow.testing:org.apache.beam.sdk.runners.dataflow.util ${dataflow.javadoc_opts}</additionalparam>
+          <additionalparam>-exclude org.apache.beam.sdk.runners.dataflow.internal:org.apache.beam.sdk.runners.dataflow.testing:org.apache.beam.sdk.runners.dataflow.util ${beam.javadoc_opts}</additionalparam>
           <use>false</use>
           <quiet>true</quiet>
           <bottom><![CDATA[<br>]]></bottom>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/sdks/java/core/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml
index aa0ad09..d44a494 100644
--- a/sdks/java/core/pom.xml
+++ b/sdks/java/core/pom.xml
@@ -60,7 +60,7 @@
             <subpackages>org.apache.beam.sdk</subpackages>
             <additionalparam>-exclude
               org.apache.beam.sdk.runners.worker:org.apache.beam.sdk.runners.dataflow:org.apache.beam.sdk.util:org.apache.beam.sdk.runners.inprocess
-              ${dataflow.javadoc_opts}</additionalparam>
+              ${beam.javadoc_opts}</additionalparam>
             <use>false</use>
             <quiet>true</quiet>
             <bottom><![CDATA[<br>]]></bottom>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/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 688d1f7..f1e3fe7 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
@@ -234,7 +234,7 @@ public interface Coder<T> extends Serializable {
    *
    * <p>If the format is changed in a backwards-compatible way (the Coder can still accept data from
    * the prior format), such as by adding optional fields to a Protocol Buffer or Avro definition,
-   * and you want Dataflow to understand that the new coder is compatible with the prior coder,
+   * and you want a Beam runner to understand that the new coder is compatible with the prior coder,
    * this value must remain unchanged. It is then the responsibility of {@link #decode} to correctly
    * read data from the prior format.
    */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/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 c5a90e0..84e59e0 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
@@ -28,7 +28,7 @@
  *
  * <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
- * chooses to execute the pipeline. As such, Dataflow requires that all PCollections have an
+ * 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
  * information and the Pipeline's {@link org.apache.beam.sdk.coders.CoderRegistry}. It

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
index 2f9054f..6091156 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
@@ -61,7 +61,7 @@ import org.slf4j.LoggerFactory;
  * <h3>Permissions</h3>
  *
  * <p>Permission requirements depend on the {@link PipelineRunner} that is used to execute the
- * Dataflow job. Please refer to the documentation of corresponding
+ * Beam pipeline. Please refer to the documentation of corresponding
  * {@link PipelineRunner PipelineRunners} for more details.
  */
 public class PubsubIO {
@@ -375,9 +375,9 @@ public class PubsubIO {
      * <p>See {@link PubsubIO.PubsubTopic#fromPath(String)} for more details on the format
      * of the {@code topic} string.
      *
-     * <p>Dataflow will start reading data published on this topic from the time the pipeline is
-     * started. Any data published on the topic before the pipeline is started will not be read by
-     * Dataflow.
+     * <p>The Beam runner will start reading data published on this topic from the time the pipeline
+     * is started. Any data published on the topic before the pipeline is started will not be read
+     * by the runner.
      */
     public static Bound<String> topic(String topic) {
       return new Bound<>(DEFAULT_PUBSUB_CODER).topic(topic);
@@ -433,9 +433,9 @@ public class PubsubIO {
      * parameter specifies the attribute name. The value of the attribute can be any string
      * that uniquely identifies this record.
      *
-     * <p>If {@code idLabel} is not provided, Dataflow cannot guarantee that no duplicate data will
-     * be delivered on the Pub/Sub stream. In this case, deduplication of the stream will be
-     * strictly best effort.
+     * <p>Pub/Sub cannot guarantee that no duplicate data will be delivered on the Pub/Sub stream.
+     * If {@code idLabel} is not provided, Beam cannot guarantee that no duplicate data will
+     * be delivered, and deduplication of the stream will be strictly best effort.
      */
     public static Bound<String> idLabel(String idLabel) {
       return new Bound<>(DEFAULT_PUBSUB_CODER).idLabel(idLabel);
@@ -830,7 +830,7 @@ public class PubsubIO {
      * representing the number of milliseconds since the Unix epoch. For example, if using the Joda
      * time classes, {@link Instant#Instant(long)} can be used to parse this value.
      *
-     * <p>If the output from this sink is being read by another Dataflow source, then
+     * <p>If the output from this sink is being read by another Beam pipeline, then
      * {@link PubsubIO.Read#timestampLabel(String)} can be used to ensure the other source reads
      * these timestamps from the appropriate attribute.
      */
@@ -843,7 +843,7 @@ public class PubsubIO {
      * published messages in an attribute with the specified name. The value of the attribute is an
      * opaque string.
      *
-     * <p>If the the output from this sink is being read by another Dataflow source, then
+     * <p>If the the output from this sink is being read by another Beam pipeline, then
      * {@link PubsubIO.Read#idLabel(String)} can be used to ensure that* the other source reads
      * these unique identifiers from the appropriate attribute.
      */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java
index 3f49eac..f256c5f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java
@@ -153,7 +153,7 @@ public abstract class Sink<T> implements Serializable, HasDisplayData {
    * parallel write to a sink as well as how to create a {@link Sink.Writer} object that can write
    * a bundle to the sink.
    *
-   * <p>Since operations in Dataflow may be run multiple times for redundancy or fault-tolerance,
+   * <p>Since operations in Beam may be run multiple times for redundancy or fault-tolerance,
    * the initialization and finalization defined by a WriteOperation <b>must be idempotent</b>.
    *
    * <p>{@code WriteOperation}s may be mutable; a {@code WriteOperation} is serialized after the

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
index 6ec4533..9d91dff 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
@@ -72,8 +72,7 @@ import org.apache.beam.sdk.values.PDone;
  * PCollection<String> lines =
  *     p.apply(TextIO.Read.from("/local/path/to/file.txt"));
  *
- * // A fully-specified Read from a GCS file (runs locally and via the
- * // Google Cloud Dataflow service):
+ * // A fully-specified Read from a GCS file:
  * PCollection<Integer> numbers =
  *     p.apply("ReadNumbers", TextIO.Read
  *         .from("gs://my_bucket/path/to/numbers-*.txt")
@@ -97,23 +96,13 @@ import org.apache.beam.sdk.values.PDone;
  * PCollection<String> lines = ...;
  * lines.apply(TextIO.Write.to("/path/to/file.txt"));
  *
- * // A fully-specified Write to a sharded GCS file (runs locally and via the
- * // Google Cloud Dataflow service):
+ * // A fully-specified Write to a sharded GCS file:
  * PCollection<Integer> numbers = ...;
  * numbers.apply("WriteNumbers", TextIO.Write
  *      .to("gs://my_bucket/path/to/numbers")
  *      .withSuffix(".txt")
  *      .withCoder(TextualIntegerCoder.of()));
  * }</pre>
- *
- * <h3>Permissions</h3>
- *
- * <p>When run using the {@code DirectRunner}, your pipeline can read and write text files
- * on your local drive and remote text files on Google Cloud Storage that you have access to using
- * your {@code gcloud} credentials. When running in the Dataflow service, the pipeline can only
- * read and write files from GCS. For more information about permissions, see the Cloud Dataflow
- * documentation on <a href="https://cloud.google.com/dataflow/security-and-permissions">Security
- * and Permissions</a>.
  */
 public class TextIO {
   /** The default coder, which returns each line of the input file as a string. */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java
index e00857e..aaee75a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java
@@ -47,13 +47,14 @@ import org.apache.beam.sdk.coders.JAXBCoder;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.values.PCollection;
 import org.codehaus.stax2.XMLInputFactory2;
 
 // CHECKSTYLE.OFF: JavadocStyle
 /**
  * A source that can be used to read XML files. This source reads one or more
- * XML files and creates a {@code PCollection} of a given type. An Dataflow read transform can be
- * created by passing an {@code XmlSource} object to {@code Read.from()}. Please note the
+ * XML files and creates a {@link PCollection} of a given type. A {@link Read} transform can be
+ * created by passing an {@link XmlSource} object to {@link Read#from}. Please note the
  * example given below.
  *
  * <p>The XML file must be of the following form, where {@code root} and {@code record} are XML
@@ -82,7 +83,7 @@ import org.codehaus.stax2.XMLInputFactory2;
  * the source will generate a {@code PCollection} of the given JAXB annotated Java type.
  * Optionally users may provide a minimum size of a bundle that should be created for the source.
  *
- * <p>The following example shows how to read from {@link XmlSource} in a Dataflow pipeline:
+ * <p>The following example shows how to read from {@link XmlSource} in a Beam pipeline:
  *
  * <pre>
  * {@code
@@ -104,13 +105,13 @@ import org.codehaus.stax2.XMLInputFactory2;
  *       such as org.codehaus.woodstox:woodstox-core-asl</li>
  * </ol>
  *
- * <p>These dependencies have been declared as optional in Maven sdk/pom.xml file of
- * Google Cloud Dataflow.
+ * <p>These dependencies have been declared as optional in the sdks/java/core/pom.xml file of
+ * Apache Beam.
  *
  * <p><h3>Permissions</h3>
  * Permission requirements depend on the
  * {@link org.apache.beam.sdk.runners.PipelineRunner PipelineRunner} that is
- * used to execute the Dataflow job. Please refer to the documentation of corresponding
+ * used to execute the Beam pipeline. Please refer to the documentation of corresponding
  * {@link PipelineRunner PipelineRunners} for more details.
  *
  * @param <T> Type of the objects that represent the records of the XML file. The

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java
index 288124b..3063441 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java
@@ -62,9 +62,9 @@ import org.slf4j.LoggerFactory;
  * <p>Key interpolation, fraction estimation, and range splitting are all interpreted in these
  * floating-point semantics. See the respective implementations for further details. <b>Note:</b>
  * the underlying implementations of these functions use {@link BigInteger} and {@link BigDecimal},
- * so they can be slow and should not be called in hot loops. Dataflow's dynamic work
- * rebalancing will only invoke these functions during periodic control operations, so they are not
- * called on the critical path.
+ * so they can be slow and should not be called in hot loops. Dynamic work rebalancing will only
+ * invoke these functions during periodic control operations, so they are not called on the critical
+ * path.
  *
  * @see ByteKey
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/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 f985a55..166c276 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
@@ -20,7 +20,7 @@
  * streaming parallel data processing
  * {@link org.apache.beam.sdk.Pipeline}s.
  *
- * <p>To use the Google Cloud Dataflow SDK, you build a
+ * <p>To use the Apache Beam SDK, you build a
  * {@link org.apache.beam.sdk.Pipeline}, which manages a graph of
  * {@link org.apache.beam.sdk.transforms.PTransform}s
  * and the {@link org.apache.beam.sdk.values.PCollection}s that

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java
index 8719384..bb6cce6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java
@@ -16,19 +16,8 @@
  * limitations under the License.
  */
 /**
- * Defines runners for executing Pipelines in different modes, including
- * {@link org.apache.beam.sdk.runners.DirectRunner} and
- * {@link org.apache.beam.sdk.runners.DataflowRunner}.
- *
- * <p>{@link org.apache.beam.sdk.runners.DirectRunner} executes a {@code Pipeline}
- * locally, without contacting the Dataflow service.
- * {@link org.apache.beam.sdk.runners.DataflowRunner} submits a
- * {@code Pipeline} to the Dataflow service, which executes it on Dataflow-managed Compute Engine
- * instances. {@code DataflowRunner} returns
- * as soon as the {@code Pipeline} has been submitted. Use
- * {@link org.apache.beam.sdk.runners.BlockingDataflowRunner} to have execution
- * updates printed to the console.
- *
- * <p>The runner is specified as part {@link org.apache.beam.sdk.options.PipelineOptions}.
+ * Defines different data processing backends (aka, runners) for executing Beam Pipelines.
+
+ * <p>The runner is specified as part of the {@link org.apache.beam.sdk.options.PipelineOptions}.
  */
 package org.apache.beam.sdk.runners;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/package-info.java
index 25b6bb2..e66677d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/package-info.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Defines utilities for unit testing Dataflow pipelines. The tests for the {@code PTransform}s and
- * examples included the Dataflow SDK provide examples of using these utilities.
+ * Defines utilities for unit testing Apache Beam pipelines. The tests for the {@code PTransform}s
+ * and examples included in the Apache Beam SDK provide examples of using these utilities.
  */
 package org.apache.beam.sdk.testing;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
index 87c7095..a445c7d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
@@ -32,6 +32,7 @@ import java.util.UUID;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.HasDisplayData;
@@ -93,7 +94,7 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
      *
      * <p>Once passed to {@code output} the element should be considered
      * immutable and not be modified in any way. It may be cached or retained
-     * by the Dataflow runtime or later steps in the pipeline, or used in
+     * by a Beam runner or later steps in the pipeline, or used in
      * other unspecified ways.
      *
      * <p>If invoked from {@link OldDoFn#processElement processElement}, the output
@@ -233,9 +234,9 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
     /**
      * Returns the input element to be processed.
      *
-     * <p>The element should be considered immutable. The Dataflow runtime will not mutate the
+     * <p>The element should be considered immutable. A Beam runner will not mutate the
      * element, so it is safe to cache, etc. The element should not be mutated by any of the
-     * {@link OldDoFn} methods, because it may be cached elsewhere, retained by the Dataflow
+     * {@link OldDoFn} methods, because it may be cached elsewhere, retained by the runner
      * runtime, or used in other unspecified ways.
      */
     public abstract InputT element();
@@ -358,14 +359,14 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
    * Processes one input element.
    *
    * <p>The current element of the input {@code PCollection} is returned by
-   * {@link ProcessContext#element() c.element()}. It should be considered immutable. The Dataflow
-   * runtime will not mutate the element, so it is safe to cache, etc. The element should not be
+   * {@link ProcessContext#element() c.element()}. It should be considered immutable. The Beam
+   * runner will not mutate the element, so it is safe to cache, etc. The element should not be
    * mutated by any of the {@link OldDoFn} methods, because it may be cached elsewhere, retained by
-   * the Dataflow runtime, or used in other unspecified ways.
+   * the Beam runner, or used in other unspecified ways.
    *
    * <p>A value is added to the main output {@code PCollection} by {@link ProcessContext#output}.
    * Once passed to {@code output} the element should be considered immutable and not be modified in
-   * any way. It may be cached elsewhere, retained by the Dataflow runtime, or used in other
+   * any way. It may be cached elsewhere, retained by the Beam runner, or used in other
    * unspecified ways.
    *
    * @see ProcessContext

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
index 2544a27..83fe577 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
@@ -125,7 +125,7 @@ import org.apache.beam.sdk.values.TypedPValue;
  * before the enclosing Pipeline is run.
  *
  * <p>A small number of PTransforms are implemented natively by the
- * Google Cloud Dataflow SDK; such PTransforms simply return an
+ * Apache Beam SDK; such PTransforms simply return an
  * output value as their apply implementation.
  * The majority of PTransforms are
  * implemented as composites of other PTransforms.  Such a PTransform

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/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 9d4c9a7..2443d8e 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
@@ -38,10 +38,9 @@ import org.apache.beam.sdk.values.TupleTagList;
 import org.apache.beam.sdk.values.TypedPValue;
 
 /**
- * {@link ParDo} is the core element-wise transform in Google Cloud
- * Dataflow, invoking a user-specified function on each of the elements of the input
- * {@link PCollection} to produce zero or more output elements, all
- * of which are collected into the output {@link PCollection}.
+ * {@link ParDo} is the core element-wise transform in Apache Beam, invoking a user-specified
+ * function on each of the elements of the input {@link PCollection} to produce zero or more output
+ * elements, all of which are collected into the output {@link PCollection}.
  *
  * <p>Elements are processed independently, and possibly in parallel across
  * distributed cloud resources.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java
index 2c0ae40..ba80de9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java
@@ -40,7 +40,7 @@ public final class ReleaseInfo extends GenericJson {
   }
 
   /**
-   * Returns an instance of DataflowReleaseInfo.
+   * Returns an instance of {@link ReleaseInfo}.
    */
   public static ReleaseInfo getReleaseInfo() {
     return LazyInit.INSTANCE;
@@ -64,13 +64,13 @@ public final class ReleaseInfo extends GenericJson {
 
     try (InputStream in = ReleaseInfo.class.getResourceAsStream(PROPERTIES_PATH)) {
       if (in == null) {
-        LOG.warn("Dataflow properties resource not found: {}", resourcePath);
+        LOG.warn("Beam properties resource not found: {}", resourcePath);
         return;
       }
 
       properties.load(in);
     } catch (IOException e) {
-      LOG.warn("Error loading Dataflow properties resource: ", e);
+      LOG.warn("Error loading Beam properties resource: ", e);
     }
 
     for (String name : properties.stringPropertyNames()) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c561cd9c/sdks/java/core/src/main/java/org/apache/beam/sdk/values/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/package-info.java
index d9bd48c..c028407 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/package-info.java
@@ -23,7 +23,7 @@
  *
  * <ul>
  *   <li>{@link org.apache.beam.sdk.values.PCollection} - an immutable collection of
- *     values of type {@code T} and the main representation for data in Dataflow.</li>
+ *     values of type {@code T} and the main representation for data in Beam.</li>
  *   <li>{@link org.apache.beam.sdk.values.PCollectionView} - an immutable view of a
  *     {@link org.apache.beam.sdk.values.PCollection} that can be accessed as a
  *     side input of a {@link org.apache.beam.sdk.transforms.ParDo}
@@ -38,7 +38,7 @@
  *     as input to {@link org.apache.beam.sdk.transforms.Flatten}.</li>
  * </ul>
  *
- * <p>And these classes for individual values play particular roles in Dataflow:
+ * <p>And these classes for individual values play particular roles in Beam:
  *
  * <ul>
  *   <li>{@link org.apache.beam.sdk.values.KV} - a key/value pair that is used by