From 74e5b7eba7fabe169d8955ecfb6c9e304ddcc442 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Mon, 26 Sep 2016 22:43:40 -0700 Subject: [PATCH 1/2] pom.xml: replace dataflow.javadoc_opts with beam.javadoc_opts --- examples/java/pom.xml | 2 +- pom.xml | 4 ++-- runners/google-cloud-dataflow-java/pom.xml | 2 +- sdks/java/core/pom.xml | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 9a48ec6b1386a..e27d101cb5f0a 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -255,7 +255,7 @@ Apache Beam Examples org.apache.beam.examples - -exclude org.apache.beam.sdk.runners.worker:org.apache.beam.sdk.runners.dataflow:org.apache.beam.sdk.util ${dataflow.javadoc_opts} + -exclude org.apache.beam.sdk.runners.worker:org.apache.beam.sdk.runners.dataflow:org.apache.beam.sdk.util ${beam.javadoc_opts} false true ]]> diff --git a/pom.xml b/pom.xml index cc93bb964a908..b262fad7dd95b 100644 --- a/pom.xml +++ b/pom.xml @@ -92,7 +92,7 @@ UTF-8 - + true @@ -223,7 +223,7 @@ [1.8,) - -Xdoclint:-missing + -Xdoclint:-missing diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index acf6ccecf60fe..7bae3669a5d7f 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -88,7 +88,7 @@ Google Cloud Dataflow Runner for Java, version ${project.version} org.apache.beam.runners.dataflow - -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} + -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} false true ]]> diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index c4d3e649be637..d285b1d7f01e7 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -60,7 +60,7 @@ org.apache.beam.sdk -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} + ${beam.javadoc_opts} false true ]]> From 3bed06939b5d16f4cf5e8c90cd867b156490aa02 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Mon, 26 Sep 2016 23:09:35 -0700 Subject: [PATCH 2/2] Minor rewording to drop Dataflow from Beam SDK core --- .../java/org/apache/beam/sdk/coders/Coder.java | 2 +- .../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 +- .../java/org/apache/beam/sdk/io/TextIO.java | 14 ++------------ .../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 +- .../apache/beam/sdk/runners/package-info.java | 17 +++-------------- .../apache/beam/sdk/testing/package-info.java | 4 ++-- .../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 +++--- .../apache/beam/sdk/values/package-info.java | 4 ++-- 15 files changed, 47 insertions(+), 67 deletions(-) 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 688d1f793ab12..f1e3fe7089d3b 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 void registerByteSizeObserver( * *

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. */ 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 c5a90e0fa5c37..84e59e018b454 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 @@ * *

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 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 d11345780295e..a47bf71f6898f 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 @@ -60,7 +60,7 @@ * *

Permissions

*

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 { @@ -374,9 +374,9 @@ public static class Read { *

See {@link PubsubIO.PubsubTopic#fromPath(String)} for more details on the format * of the {@code topic} string. * - *

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. + *

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 topic(String topic) { return new Bound<>(DEFAULT_PUBSUB_CODER).topic(topic); @@ -432,9 +432,9 @@ public static Bound timestampLabel(String timestampLabel) { * parameter specifies the attribute name. The value of the attribute can be any string * that uniquely identifies this record. * - *

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. + *

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 idLabel(String idLabel) { return new Bound<>(DEFAULT_PUBSUB_CODER).idLabel(idLabel); @@ -829,7 +829,7 @@ public static Bound topic(String topic) { * 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. * - *

If the output from this sink is being read by another Dataflow source, then + *

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. */ @@ -842,7 +842,7 @@ public static Bound timestampLabel(String timestampLabel) { * published messages in an attribute with the specified name. The value of the attribute is an * opaque string. * - *

If the the output from this sink is being read by another Dataflow source, then + *

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. */ 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 1abcc3d1166d5..e01cb041d142a 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 @@ -150,7 +150,7 @@ public void populateDisplayData(DisplayData.Builder builder) {} * parallel write to a sink as well as how to create a {@link Sink.Writer} object that can write * a bundle to the sink. * - *

Since operations in Dataflow may be run multiple times for redundancy or fault-tolerance, + *

Since operations in Beam may be run multiple times for redundancy or fault-tolerance, * the initialization and finalization defined by a WriteOperation must be idempotent. * *

{@code WriteOperation}s may be mutable; a {@code WriteOperation} is serialized after the 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 62d3ae80f7c3f..9ff33c9648775 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 @@ * PCollection 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 numbers = * p.apply("ReadNumbers", TextIO.Read * .from("gs://my_bucket/path/to/numbers-*.txt") @@ -97,22 +96,13 @@ * PCollection 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 numbers = ...; * numbers.apply("WriteNumbers", TextIO.Write * .to("gs://my_bucket/path/to/numbers") * .withSuffix(".txt") * .withCoder(TextualIntegerCoder.of())); * } - * - *

Permissions

- *

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 Security - * and Permissions. */ public class TextIO { /** The default coder, which returns each line of the input file as a string. */ 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 e00857ea2bf82..aaee75aac6ab1 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.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. * *

The XML file must be of the following form, where {@code root} and {@code record} are XML @@ -82,7 +83,7 @@ * 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. * - *

The following example shows how to read from {@link XmlSource} in a Dataflow pipeline: + *

The following example shows how to read from {@link XmlSource} in a Beam pipeline: * *

  * {@code
@@ -104,13 +105,13 @@
  *       such as org.codehaus.woodstox:woodstox-core-asl
  * 
  *
- * 

These dependencies have been declared as optional in Maven sdk/pom.xml file of - * Google Cloud Dataflow. + *

These dependencies have been declared as optional in the sdks/java/core/pom.xml file of + * Apache Beam. * *

Permissions

* 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 Type of the objects that represent the records of the XML file. The 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 288124b310ecc..30634411255e3 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 @@ *

Key interpolation, fraction estimation, and range splitting are all interpreted in these * floating-point semantics. See the respective implementations for further details. Note: * 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 */ 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 f985a558a3f97..166c27651dc21 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. * - *

To use the Google Cloud Dataflow SDK, you build a + *

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 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 8719384ce228f..bb6cce6079cb0 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}. - * - *

{@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. - * - *

The runner is specified as part {@link org.apache.beam.sdk.options.PipelineOptions}. + * Defines different data processing backends (aka, runners) for executing Beam Pipelines. + + *

The runner is specified as part of the {@link org.apache.beam.sdk.options.PipelineOptions}. */ package org.apache.beam.sdk.runners; 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 25b6bb28b0ade..e66677df86ab0 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; 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 474efef2c3d36..acd1203463ae1 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 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 Context { * *

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. * *

If invoked from {@link OldDoFn#processElement processElement}, the output @@ -233,9 +234,9 @@ public abstract class ProcessContext extends Context { /** * Returns the input element to be processed. * - *

The element should be considered immutable. The Dataflow runtime will not mutate the + *

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 void startBundle(Context c) throws Exception { * Processes one input element. * *

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. * *

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 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 4a58141ab503f..9c22f2555ebe5 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 @@ * before the enclosing Pipeline is run. * *

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 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 f9cb557c1e5df..a3d27d0c4e255 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.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}. * *

Elements are processed independently, and possibly in parallel across * distributed cloud resources. 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 2c0ae4099c804..ba80de97dc4de 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 @@ private static class LazyInit { } /** - * Returns an instance of DataflowReleaseInfo. + * Returns an instance of {@link ReleaseInfo}. */ public static ReleaseInfo getReleaseInfo() { return LazyInit.INSTANCE; @@ -64,13 +64,13 @@ private ReleaseInfo(String resourcePath) { 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()) { 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 d9bd48ce56a6c..c0284078df741 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 @@ * *

    *
  • {@link org.apache.beam.sdk.values.PCollection} - an immutable collection of - * values of type {@code T} and the main representation for data in Dataflow.
  • + * values of type {@code T} and the main representation for data in Beam. *
  • {@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}.
  • *
* - *

And these classes for individual values play particular roles in Dataflow: + *

And these classes for individual values play particular roles in Beam: * *

    *
  • {@link org.apache.beam.sdk.values.KV} - a key/value pair that is used by