Rewrites the section on Coders to not talk about them as a parsing mechanism


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

Branch: refs/heads/asf-site
Commit: 0d0da0265d8a3ee07493feec835e56efd6acfd85
Parents: 9cc5b22
Author: Eugene Kirpichov <kirpic...@google.com>
Authored: Fri May 12 16:06:09 2017 -0700
Committer: Eugene Kirpichov <kirpic...@google.com>
Committed: Mon May 15 11:28:52 2017 -0700

----------------------------------------------------------------------
 src/documentation/programming-guide.md | 38 ++++++-----------------------
 1 file changed, 8 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam-site/blob/0d0da026/src/documentation/programming-guide.md
----------------------------------------------------------------------
diff --git a/src/documentation/programming-guide.md 
b/src/documentation/programming-guide.md
index 11ec86d..f70e255 100644
--- a/src/documentation/programming-guide.md
+++ b/src/documentation/programming-guide.md
@@ -1175,11 +1175,9 @@ See the  [Beam-provided I/O Transforms]({{site.baseurl 
}}/documentation/io/built
 
 ## <a name="coders"></a>Data encoding and type safety
 
-When you create or output pipeline data, you'll need to specify how the 
elements in your `PCollection`s are encoded and decoded to and from byte 
strings. Byte strings are used for intermediate storage as well reading from 
sources and writing to sinks. The Beam SDKs use objects called coders to 
describe how the elements of a given `PCollection` should be encoded and 
decoded.
+When Beam runners execute your pipeline, they often need to materialize the 
intermediate data in your `PCollection`s, which requires converting elements to 
and from byte strings. The Beam SDKs use objects called `Coder`s to describe 
how the elements of a given `PCollection` may be encoded and decoded.
 
-### Using coders
-
-You typically need to specify a coder when reading data into your pipeline 
from an external source (or creating pipeline data from local data), and also 
when you output pipeline data to an external sink.
+> Note that coders are unrelated to parsing or formatting data when 
interacting with external data sources or sinks. Such parsing or formatting 
should typically be done explicitly, using transforms such as `ParDo` or 
`MapElements`.
 
 {:.language-java}
 In the Beam SDK for Java, the type `Coder` provides the methods required for 
encoding and decoding data. The SDK for Java provides a number of Coder 
subclasses that work with a variety of standard Java types, such as Integer, 
Long, Double, StringUtf8 and more. You can find all of the available Coder 
subclasses in the [Coder 
package](https://github.com/apache/beam/tree/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders).
@@ -1187,38 +1185,18 @@ In the Beam SDK for Java, the type `Coder` provides the 
methods required for enc
 {:.language-py}
 In the Beam SDK for Python, the type `Coder` provides the methods required for 
encoding and decoding data. The SDK for Python provides a number of Coder 
subclasses that work with a variety of standard Python types, such as primitive 
types, Tuple, Iterable, StringUtf8 and more. You can find all of the available 
Coder subclasses in the 
[apache_beam.coders](https://github.com/apache/beam/tree/master/sdks/python/apache_beam/coders)
 package.
 
-When you read data into a pipeline, the coder indicates how to interpret the 
input data into a language-specific type, such as integer or string. Likewise, 
the coder indicates how the language-specific types in your pipeline should be 
written into byte strings for an output data sink, or to materialize 
intermediate data in your pipeline.
-
-The Beam SDKs set a coder for every `PCollection` in a pipeline, including 
those generated as output from a transform. Most of the time, the Beam SDKs can 
automatically infer the correct coder for an output `PCollection`.
-
 > Note that coders do not necessarily have a 1:1 relationship with types. For 
 > example, the Integer type can have multiple valid coders, and input and 
 > output data can use different Integer coders. A transform might have 
 > Integer-typed input data that uses BigEndianIntegerCoder, and Integer-typed 
 > output data that uses VarIntCoder.
 
-You can explicitly set a `Coder` when inputting or outputting a `PCollection`. 
You set the `Coder` by <span class="language-java">calling the method 
`.withCoder`</span> <span class="language-py">setting the `coder` 
argument</span> when you apply your pipeline's read or write transform.
-
-Typically, you set the `Coder` when the coder for a `PCollection` cannot be 
automatically inferred, or when you want to use a different coder than your 
pipeline's default. The following example code reads a set of numbers from a 
text file, and sets a `Coder` of type <span 
class="language-java">`TextualIntegerCoder`</span> <span 
class="language-py">`VarIntCoder`</span> for the resulting `PCollection`:
-
-```java
-PCollection<Integer> numbers =
-  p.begin()
-  .apply(TextIO.Read.named("ReadNumbers")
-    .from("gs://my_bucket/path/to/numbers-*.txt")
-    .withCoder(TextualIntegerCoder.of()));
-```
-
-```py
-p = beam.Pipeline()
-numbers = ReadFromText("gs://my_bucket/path/to/numbers-*.txt", 
coder=VarIntCoder())
-```
+### Specifying coders
+The Beam SDKs require a coder for every `PCollection` in your pipeline. In 
most cases, the Beam SDK is able to automatically infer a `Coder` for a 
`PCollection` based on its element type or the transform that produces it, 
however, in some cases the pipeline author will need to specify a `Coder` 
explicitly, or develop a `Coder` for their custom type.
 
 {:.language-java}
-You can set the coder for an existing `PCollection` by using the method 
`PCollection.setCoder`. Note that you cannot call `setCoder` on a `PCollection` 
that has been finalized (e.g. by calling `.apply` on it).
+You can explicitly set the coder for an existing `PCollection` by using the 
method `PCollection.setCoder`. Note that you cannot call `setCoder` on a 
`PCollection` that has been finalized (e.g. by calling `.apply` on it).
 
 {:.language-java}
-You can get the coder for an existing `PCollection` by using the method 
`getCoder`. This method will fail with `anIllegalStateException` if a coder has 
not been set and cannot be inferred for the given `PCollection`.
-
-### Coder inference and default coders
+You can get the coder for an existing `PCollection` by using the method 
`getCoder`. This method will fail with an `IllegalStateException` if a coder 
has not been set and cannot be inferred for the given `PCollection`.
 
-The Beam SDKs require a coder for every `PCollection` in your pipeline. Most 
of the time, however, you do not need to explicitly specify a coder, such as 
for an intermediate `PCollection` produced by a transform in the middle of your 
pipeline. In such cases, the Beam SDKs can infer an appropriate coder from the 
inputs and outputs of the transform used to produce the PCollection.
+Beam SDKs use a variety of mechanisms when attempting to automatically infer 
the `Coder` for a `PCollection`.
 
 {:.language-java}
 Each pipeline object has a `CoderRegistry`. The `CoderRegistry` represents a 
mapping of Java types to the default coders that the pipeline should use for 
`PCollection`s of each type.
@@ -1227,7 +1205,7 @@ Each pipeline object has a `CoderRegistry`. The 
`CoderRegistry` represents a map
 The Beam SDK for Python has a `CoderRegistry` that represents a mapping of 
Python types to the default coder that should be used for `PCollection`s of 
each type.
 
 {:.language-java}
-By default, the Beam SDK for Java automatically infers the `Coder` for the 
elements of an output `PCollection` using the type parameter from the 
transform's function object, such as `DoFn`. In the case of `ParDo`, for 
example, a `DoFn<Integer, String>function` object accepts an input element of 
type `Integer` and produces an output element of type `String`. In such a case, 
the SDK for Java will automatically infer the default `Coder` for the output 
`PCollection<String>` (in the default pipeline `CoderRegistry`, this is 
`StringUtf8Coder`).
+By default, the Beam SDK for Java automatically infers the `Coder` for the 
elements of a `PCollection` produced by a `PTransform` using the type parameter 
from the transform's function object, such as `DoFn`. In the case of `ParDo`, 
for example, a `DoFn<Integer, String>` function object accepts an input element 
of type `Integer` and produces an output element of type `String`. In such a 
case, the SDK for Java will automatically infer the default `Coder` for the 
output `PCollection<String>` (in the default pipeline `CoderRegistry`, this is 
`StringUtf8Coder`).
 
 {:.language-py}
 By default, the Beam SDK for Python automatically infers the `Coder` for the 
elements of an output `PCollection` using the typehints from the transform's 
function object, such as `DoFn`. In the case of `ParDo`, for example a `DoFn` 
with the typehints `@beam.typehints.with_input_types(int)` and 
`@beam.typehints.with_output_types(str)` accepts an input element of type int 
and produces an output element of type str. In such a case, the Beam SDK for 
Python will automatically infer the default `Coder` for the output 
`PCollection` (in the default pipeline `CoderRegistry`, this is `BytesCoder`).

Reply via email to