beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From fran...@apache.org
Subject [1/3] beam-site git commit: Update programming guide with Python snippets and data encoding section.
Date Wed, 08 Feb 2017 18:55:05 GMT
Repository: beam-site
Updated Branches:
  refs/heads/asf-site e1dd10a94 -> 221f388d3


Update programming guide with Python snippets and data encoding section.


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

Branch: refs/heads/asf-site
Commit: f277339244e52c4fc40c6de1829d746964f844cc
Parents: e1dd10a
Author: Hadar Hod <hadarh@google.com>
Authored: Fri Feb 3 16:35:41 2017 -0800
Committer: Frances Perry <fjp@google.com>
Committed: Wed Feb 8 10:41:58 2017 -0800

----------------------------------------------------------------------
 src/documentation/programming-guide.md | 341 ++++++++++++++++++++++------
 1 file changed, 267 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam-site/blob/f2773392/src/documentation/programming-guide.md
----------------------------------------------------------------------
diff --git a/src/documentation/programming-guide.md b/src/documentation/programming-guide.md
index 3cd64f0..aaf5cdb 100644
--- a/src/documentation/programming-guide.md
+++ b/src/documentation/programming-guide.md
@@ -55,7 +55,7 @@ The Beam SDKs provide a number of abstractions that simplify the mechanics
of la
 
 * `PCollection`: A `PCollection` represents a distributed data set that your Beam pipeline
operates on. The data set can be *bounded*, meaning it comes from a fixed source like a file,
or *unbounded*, meaning it comes from a continuously updating source via a subscription or
other mechanism. Your pipeline typically creates an initial `PCollection` by reading data
from an external data source, but you can also create a `PCollection` from in-memory data
within your driver program. From there, `PCollection`s are the inputs and outputs for each
step in your pipeline.
 
-* `Transform`: A `Transform` represents a data processing operation, or a step, in your pipeline.
Every `Transform` takes one or more `PCollection` objects as input, performs a processing
function that you provide on the elements of that `PCollection`, and produces one or more
output `PCollection` objects. 
+* `Transform`: A `Transform` represents a data processing operation, or a step, in your pipeline.
Every `Transform` takes one or more `PCollection` objects as input, perfroms a processing
function that you provide on the elements of that `PCollection`, and produces one or more
output `PCollection` objects. 
 
 * I/O `Source` and `Sink`: Beam provides `Source` and `Sink` APIs to represent reading and
writing data, respectively. `Source` encapsulates the code necessary to read data into your
Beam pipeline from some external source, such as cloud file storage or a subscription to a
streaming data source. `Sink` likewise encapsulates the code necessary to write the elements
of a `PCollection` to an external data sink.
 
@@ -91,11 +91,11 @@ public static void main(String[] args) {
 ```
 
 ```py
-from apache_beam.utils.pipeline_options import PipelineOptions
-
-# Will parse the arguments passed into the application and construct a PipelineOptions
+# Will parse the arguments passed into the application and construct a PipelineOptions object.
 # Note that --help will print registered options.
-p = beam.Pipeline(options=PipelineOptions())
+
+{% github_sample /apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py
tag:pipelines_constructing_creating
+%}
 ```
 
 The Beam SDKs contain various subclasses of `PipelineOptions` that correspond to different
Runners. For example, `DirectPipelineOptions` contains options for the Direct (local) pipeline
runner, while `DataflowPipelineOptions` contains options for using the runner for Google Cloud
Dataflow. You can also define your own custom `PipelineOptions` by creating an interface that
extends the Beam SDKs' `PipelineOptions` class.
@@ -129,16 +129,10 @@ public static void main(String[] args) {
 ```
 
 ```py
-import apache_beam as beam
-
-# Create the pipeline.
-p = beam.Pipeline()
-
-# Read the text file into a PCollection.
-lines = p | 'ReadMyFile' >> beam.io.Read(beam.io.TextFileSource("protocol://path/to/some/inputData.txt"))
+{% github_sample /apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py
tag:pipelines_constructing_reading
+%}
 ```
 
-
 See the [section on I/O](#io) to learn more about how to read from the various data sources
supported by the Beam SDK.
 
 #### Creating a PCollection from in-memory data
@@ -174,20 +168,8 @@ public static void main(String[] args) {
 ```
 
 ```py
-import apache_beam as beam
-
-# python list
-lines = [
-  "To be, or not to be: that is the question: ",
-  "Whether 'tis nobler in the mind to suffer ",
-  "The slings and arrows of outrageous fortune, ",
-  "Or to take arms against a sea of troubles, "
-]
-
-# Create the pipeline.
-p = beam.Pipeline()
-
-collection = p | 'ReadMyLines' >> beam.Create(lines)
+{% github_sample /apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py
tag:model_pcollection
+%}
 ```
 
 ### <a name="pccharacteristics"></a>PCollection characteristics
@@ -246,8 +228,8 @@ How you apply your pipeline's transforms determines the structure of your
pipeli
 
 ```java
 [Final Output PCollection] = [Initial Input PCollection].apply([First Transform])
-							.apply([Second Transform])
-							.apply([Third Transform])
+.apply([Second Transform])
+.apply([Third Transform])
 ```
 
 ```py
@@ -262,16 +244,21 @@ The resulting workflow graph of the above pipeline looks like this:
 
 However, note that a transform *does not consume or otherwise alter* the input collection--remember
that a `PCollection` is immutable by definition. This means that you can apply multiple transforms
to the same input `PCollection` to create a branching pipeline, like so:
 
-```
+```java
 [Output PCollection 1] = [Input PCollection].apply([Transform 1])
 [Output PCollection 2] = [Input PCollection].apply([Transform 2])
 ```
 
+```py
+[Output PCollection 1] = [Input PCollection] | [Transform 1]
+[Output PCollection 2] = [Input PCollection] | [Transform 2]
+```
+
 The resulting workflow graph from the branching pipeline above looks like this:
 
 [Branching Graph Graphic]
 
-You can also build your own [composite transforms](#transforms-composite) that nest multiple
sub-steps inside a single, larger transform. Composite transforms are particularly useful
for building a reusable sequence of simple steps that get used in a lot of different places.
+You can also build your own composite transforms that nest multiple sub-steps inside a single,
larger transform. Composite transforms are particularly useful for building a reusable sequence
of simple steps that get used in a lot of different places.
 
 ### Transforms in the Beam SDK
 
@@ -326,13 +313,8 @@ PCollection<Integer> wordLengths = words.apply(
 words = ...
 
 # The DoFn to perform on each element in the input PCollection.
-class ComputeWordLengthFn(beam.DoFn):
-  def process(self, context):
-    # Get the input element from ProcessContext.
-    word = context.element
-    # Use return to emit the output element.
-    return [len(word)]
-
+{% github_sample /apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py
tag:model_pardo_pardo
+%}    
 {% github_sample /apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py
tag:model_pardo_apply
 %}```
 
@@ -370,12 +352,8 @@ static class ComputeWordLengthFn extends DoFn<String, Integer>
{
 ```
 
 ```py
-class ComputeWordLengthFn(beam.DoFn):
-  def process(self, context):
-    # Get the input element from ProcessContext.
-    word = context.element
-    # Use return to emit the output element.
-    return [len(word)]
+{% github_sample /apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py
tag:model_pardo_pardo
+%}
 ```
 
 {:.language-java}
@@ -591,8 +569,8 @@ PCollection<Integer> sum = pc.apply(
 # sum combines the elements in the input PCollection.
 # The resulting PCollection, called result, contains one value: the sum of all the elements
in the input PCollection.
 pc = ...
-result = pc | beam.CombineGlobally(sum)
-```
+{% github_sample /apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py
tag:combine_custom_average
+%}```
 
 ##### Global windowing:
 
@@ -609,7 +587,6 @@ PCollection<Integer> sum = pc.apply(
 ```py
 pc = ...
 sum = pc | beam.CombineGlobally(sum).without_defaults()
-
 ```
 
 ##### Non-global windowing:
@@ -654,9 +631,8 @@ PCollection<KV<String, Double>> avgAccuracyPerPlayer =
 ```py
 # PCollection is grouped by key and the numeric values associated with each key are averaged
into a float.
 player_accuracies = ...
-avg_accuracy_per_player = (player_accuracies
-                           | beam.CombinePerKey(
-                               beam.combiners.MeanCombineFn()))
+{% github_sample /apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py
tag:combine_per_key
+%}
 ```
 
 #### <a name="transforms-flatten-partition"></a>Using Flatten and Partition
@@ -680,11 +656,10 @@ PCollection<String> merged = collections.apply(Flatten.<String>pCollections());
 
 ```py
 # Flatten takes a tuple of PCollection objects.
-# Returns a single PCollection that contains all of the elements in the PCollection objects
in that tuple.
-merged = (
-    (pcoll1, pcoll2, pcoll3)
-    # A list of tuples can be "piped" directly into a Flatten transform.
-    | beam.Flatten())
+# Returns a single PCollection that contains all of the elements in the 
+{%
+github_sample /apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py
tag:model_multiple_pcollections_flatten
+%}
 ```
 
 ##### Data encoding in merged collections:
@@ -723,13 +698,12 @@ PCollection<Student> fortiethPercentile = studentsByPercentile.get(4);
 ```py
 # Provide an int value with the desired number of result partitions, and a partitioning function
(partition_fn in this example).
 # Returns a tuple of PCollection objects containing each of the resulting partitions as individual
PCollection objects.
-def partition_fn(student, num_partitions):
-  return int(get_percentile(student) * num_partitions / 100)
-
-by_decile = students | beam.Partition(partition_fn, 10)
+{% github_sample /apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py
tag:model_multiple_pcollections_partition
+%}
 
 # You can extract each partition from the tuple of PCollection objects as follows:
-fortieth_percentile = by_decile[4]
+{% github_sample /apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py
tag:model_multiple_pcollections_partition_40th
+%}
 ```
 
 #### <a name="transforms-usercodereqs"></a>General Requirements for writing user
code for Beam transforms
@@ -957,7 +931,7 @@ Read transforms read data from an external source and return a `PCollection`
rep
 PCollection<String> lines = p.apply(TextIO.Read.from("gs://some/inputData.txt")); 
 
 ```
 
-```python
+```py
 lines = pipeline | beam.io.ReadFromText('gs://some/inputData.txt')
 ```
 
@@ -971,7 +945,7 @@ Write transforms write the data in a `PCollection` to an external data
source. Y
 output.apply(TextIO.Write.to("gs://some/outputData"));
 ```
 
-```python
+```py
 output | beam.io.WriteToText('gs://some/outputData')
 ```
 
@@ -986,10 +960,9 @@ p.apply(“ReadFromText”,
     TextIO.Read.from("protocol://my_bucket/path/to/input-*.csv");
 ```
 
-```python
-lines = p | beam.io.Read(
-    'ReadFromText',
-    beam.io.TextFileSource('protocol://my_bucket/path/to/input-*.csv'))
+```py
+{% github_sample /apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py
tag:model_pipelineio_read
+%}
 ```
 
 To read data from disparate sources into a single `PCollection`, read each one independently
and then use the [Flatten](#transforms-flatten-partition) transform to create a single `PCollection`.
@@ -1006,9 +979,9 @@ records.apply("WriteToText",
                 .withSuffix(".csv"));
 ```
 
-```python
-filtered_words | beam.io.WriteToText(
-'protocol://my_bucket/path/to/numbers', file_name_suffix='.csv')
+```py
+{% github_sample /apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py
tag:model_pipelineio_write
+%}
 ```
 
 ### Beam-provided I/O APIs
@@ -1069,22 +1042,242 @@ To run your pipeline, use the `run` method. The program you create
sends a speci
 pipeline.run();
 ```
 
-```python
+```py
 pipeline.run()
 ```
 
-For blocking execution, append the `waitUntilFinish` method:
+For blocking execution, append the <span class="language-java">`waitUntilFinish`</span>
<span class="language-py">`wait_until_finish`</span> method:
 
 ```java
 pipeline.run().waitUntilFinish();
 ```
 
-```python
+```py
 pipeline.run().wait_until_finish()
 ```
 
-<a name="transforms-composite"></a>
-<a name="coders"></a>
+## <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.
+
+### 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.
+
+{:.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).
+
+{:.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())
+```
+
+{:.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).
+
+{:.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
+
+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.
+
+{:.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.
+
+{:.language-py}
+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`).
+
+{:.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`).
+
+> NOTE: If you create your `PCollection` from in-memory data by using the `Create` transform,
you cannot rely on coder inference and default coders. `Create` does not have access to any
typing information for its arguments, and may not be able to infer a coder if the argument
list contains a value whose exact run-time class doesn't have a default coder registered.

+
+{:.language-java}
+When using `Create`, the simplest way to ensure that you have the correct coder is by invoking
`withCoder` when you apply the `Create` transform.
+
+#### Default coders and the CoderRegistry
+
+Each Pipeline object has a `CoderRegistry` object, which maps language types to the default
coder the pipeline should use for those types. You can use the `CoderRegistry` yourself to
look up the default coder for a given type, or to register a new default coder for a given
type.
+
+`CoderRegistry` contains a default mapping of coders to standard <span class="language-java">Java</span>
<span class="language-py">Python</span> types for any pipeline you create using
the Beam SDK for <span class="language-java">Java</span> <span class="language-py">Python</span>.
The following table shows the standard mapping:
+
+{:.language-java}
+<table>
+  <thead>
+    <tr class="header">
+      <th>Java Type</th>
+      <th>Default Coder</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr class="odd">
+      <td>Double</td>
+      <td>DoubleCoder</td>
+    </tr>
+    <tr class="even">
+      <td>Instant</td>
+      <td>InstantCoder</td>
+    </tr>
+    <tr class="odd">
+      <td>Integer</td>
+      <td>VarIntCoder</td>
+    </tr>
+    <tr class="even">
+      <td>Iterable</td>
+      <td>IterableCoder</td>
+    </tr>
+    <tr class="odd">
+      <td>KV</td>
+      <td>KvCoder</td>
+    </tr>
+    <tr class="even">
+      <td>List</td>
+      <td>ListCoder</td>
+    </tr>
+    <tr class="odd">
+      <td>Map</td>
+      <td>MapCoder</td>
+    </tr>
+    <tr class="even">
+      <td>Long</td>
+      <td>VarLongCoder</td>
+    </tr>
+    <tr class="odd">
+      <td>String</td>
+      <td>StringUtf8Coder</td>
+    </tr>
+    <tr class="even">
+      <td>TableRow</td>
+      <td>TableRowJsonCoder</td>
+    </tr>
+    <tr class="odd">
+      <td>Void</td>
+      <td>VoidCoder</td>
+    </tr>
+    <tr class="even">
+      <td>byte[ ]</td>
+      <td>ByteArrayCoder</td>
+    </tr>
+    <tr class="odd">
+      <td>TimestampedValue</td>
+      <td>TimestampedValueCoder</td>
+    </tr>
+  </tbody>
+</table>
+
+{:.language-py}
+<table>
+  <thead>
+    <tr class="header">
+      <th>Python Type</th>
+      <th>Default Coder</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr class="odd">
+      <td>int</td>
+      <td>VarIntCoder</td>
+    </tr>
+    <tr class="even">
+      <td>float</td>
+      <td>FloatCoder</td>
+    </tr>
+    <tr class="odd">
+      <td>str</td>
+      <td>BytesCoder</td>
+    </tr>
+    <tr class="even">
+      <td>bytes</td>
+      <td>StrUtf8Coder</td>
+    </tr>
+    <tr class="odd">
+      <td>Tuple</td>
+      <td>TupleCoder</td>
+    </tr>
+  </tbody>
+</table>
+
+##### Looking up a default coder
+
+{:.language-java}
+You can use the method `CoderRegistry.getDefaultCoder` to determine the default Coder for
a Java type. You can access the `CoderRegistry` for a given pipeline by using the method `Pipeline.getCoderRegistry`.
This allows you to determine (or set) the default Coder for a Java type on a per-pipeline
basis: i.e. "for this pipeline, verify that Integer values are encoded using `BigEndianIntegerCoder`."
+
+{:.language-py}
+You can use the method `CoderRegistry.get_coder` to determine the default Coder for a Python
type. You can use `coders.registry` to access the `CoderRegistry`. This allows you to determine
(or set) the default Coder for a Python type.
+
+##### Setting the default coder for a type
+
+To set the default Coder for a <span class="language-java">Java</span> <span
class="language-py">Python</span> type for a particular pipeline, you obtain and
modify the pipeline's `CoderRegistry`. You use the method <span class="language-java">`Pipeline.getCoderRegistry`</span>
<span class="language-py">`coders.registry`</span> to get the `CoderRegistry`
object, and then use the method <span class="language-java">`CoderRegistry.registerCoder`</span>
<span class="language-py">`CoderRegistry.register_coder`</span> to register a
new `Coder` for the target type.
+
+The following example code demonstrates how to set a default Coder, in this case `BigEndianIntegerCoder`,
for <span class="language-java">Integer</span> <span class="language-py">int</span>
values for a pipeline.
+
+```java  
+PipelineOptions options = PipelineOptionsFactory.create();
+Pipeline p = Pipeline.create(options);
+
+CoderRegistry cr = p.getCoderRegistry();
+cr.registerCoder(Integer.class, BigEndianIntegerCoder.class);
+```
+
+```py
+apache_beam.coders.registry.register_coder(int, BigEndianIntegerCoder)
+```
+
+##### Annotating a custom data type with a default coder
+
+{:.language-java}
+If your pipeline program defines a custom data type, you can use the `@DefaultCoder` annotation
to specify the coder to use with that type. For example, let's say you have a custom data
type for which you want to use `SerializableCoder`. You can use the `@DefaultCoder` annotation
as follows:
+
+```java
+@DefaultCoder(AvroCoder.class)
+public class MyCustomDataType {
+  ...
+}
+```
+
+{:.language-java}
+If you've created a custom coder to match your data type, and you want to use the `@DefaultCoder`
annotation, your coder class must implement a static `Coder.of(Class<T>)` factory method.
+
+```java
+public class MyCustomCoder implements Coder {
+  public static Coder<T> of(Class<T> clazz) {...}
+  ...
+}
+
+@DefaultCoder(MyCustomCoder.class)
+public class MyCustomDataType {
+  ...
+}
+```
+
+{:.language-py}
+The Beam SDK for Python does not support annotating data types with a default coder. If you
would like to set a default coder, use the method described in the previous section, *Setting
the default coder for a type*.
+
 <a name="windowing"></a>
 <a name="triggers"></a>
 


Mime
View raw message