A
PTransform is an operation that takes an
InputT (some subtype
of
PInput) and produces an
OutputT (some subtype of
POutput).
Common PTransforms include root PTransforms like
org.apache.beam.sdk.io.TextIO.Read,
Create, processing and conversion operations like
ParDo,
GroupByKey,
org.apache.beam.sdk.transforms.join.CoGroupByKey,
Combine, and
Count, and
outputting PTransforms like
org.apache.beam.sdk.io.TextIO.Write. Users also define their
own application-specific composite PTransforms.
Each
PTransform has a single
InputT type and a single
OutputT type. Many PTransforms conceptually transform one input value to one output value, and
in this case
InputT and
Output are typically instances of
org.apache.beam.sdk.values.PCollection. A root PTransform conceptually has no input; in this
case, conventionally a
org.apache.beam.sdk.values.PBegin object produced by calling
Pipeline#begin is used as the input. An outputting PTransform conceptually has no output;
in this case, conventionally
org.apache.beam.sdk.values.PDone is used as its output type.
Some PTransforms conceptually have multiple inputs and/or outputs; in these cases special
"bundling" classes like
org.apache.beam.sdk.values.PCollectionList,
org.apache.beam.sdk.values.PCollectionTuple are used to combine multiple values into a single
bundle for passing into or returning from the PTransform.
A
PTransform is invoked by calling
apply() on its
InputT, returning its
OutputT. Calls can be chained to concisely create linear pipeline
segments. For example:
PCollection pc1 = ...;
PTransform operations have unique names, which are used by the system when explaining what's
going on during optimization and execution. Each PTransform gets a system-provided default name,
but it's a good practice to specify a more informative explicit name when applying the transform.
For example:
...
Each PCollection output produced by a PTransform, either directly or within a "bundling"
class, automatically gets its own name derived from the name of its producing PTransform.
Each PCollection output produced by a PTransform also records a
org.apache.beam.sdk.coders.Coder that specifies how the elements of that PCollection are to be
encoded as a byte string, if necessary. The PTransform may provide a default Coder for any of its
outputs, for instance by deriving it from the PTransform input's Coder. If the PTransform does
not specify the Coder for an output PCollection, the system will attempt to infer a Coder for it,
based on what's known at run-time about the Java type of the output's elements. The enclosing
Pipeline's
org.apache.beam.sdk.coders.CoderRegistry (accessible via
Pipeline#getCoderRegistry) defines the mapping from Java types to the default Coder to use, for
a standard set of Java types; users can extend this mapping for additional types, via
org.apache.beam.sdk.coders.CoderRegistry#registerCoderProvider. If this inference process fails,
either because the Java type was not known at run-time (e.g., due to Java's "erasure" of generic
types) or there was no default Coder registered, then the Coder should be specified manually by
calling
PCollection#setCoder on the output PCollection. The Coder of every output
PCollection must be determined one way or another before that output is used as an input to
another PTransform, or before the enclosing Pipeline is run.
A small number of PTransforms are implemented natively by the 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 subclass
typically just implements
#expand, computing its Output value from its
InputTvalue. User programs are encouraged to use this mechanism to modularize their own code. Such
composite abstractions get their own name, and navigating through the composition hierarchy of
PTransforms is supported by the monitoring interface. Examples of composite PTransforms can be
found in this directory and in examples. From the caller's point of view, there is no distinction
between a PTransform implemented natively and one implemented in terms of other PTransforms; both
kinds of PTransform are invoked in the same way, using
apply().
Note on Serialization
PTransform doesn't actually support serialization, despite implementing
Serializable.
PTransform is marked
Serializable solely because it is common for an anonymous
DoFn, instance to be created within an
apply() method of a composite
PTransform.
Each of those
*Fns is
Serializable, but unfortunately its instance state will
contain a reference to the enclosing
PTransform instance, and so attempt to serialize the
PTransform instance, even though the
*Fn instance never references anything about
the enclosing
PTransform.
To allow such anonymous
*Fns to be written conveniently,
PTransform is marked
as
Serializable, and includes dummy
writeObject() and
readObject()operations that do not save or restore any state.