InputT
- the type of the input to this PTransformOutputT
- the type of the output of this PTransformpublic abstract class PTransform<InputT extends PInput,OutputT extends POutput> extends java.lang.Object implements java.io.Serializable, HasDisplayData
PTransform<InputT, OutputT>
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 TextIO.Read
,
Create
, processing and conversion operations like ParDo
, GroupByKey
,
CoGroupByKey
, Combine
, and Count
, and
outputting PTransforms like TextIO.Write
. Users also define their
own application-specific composite PTransforms.
Each PTransform<InputT, OutputT>
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 PCollection
. A root PTransform conceptually has no input; in this
case, conventionally a PBegin
object produced by calling
Pipeline.begin()
is used as the input. An outputting PTransform conceptually has no output;
in this case, conventionally PDone
is used as its output type.
Some PTransforms conceptually have multiple inputs and/or outputs; in these cases special
"bundling" classes like PCollectionList
, PCollectionTuple
are used to combine multiple values into a single
bundle for passing into or returning from the PTransform.
A PTransform<InputT, OutputT>
is invoked by calling apply()
on its InputT
, returning its OutputT
. Calls can be chained to concisely create linear pipeline
segments. For example:
PCollection<T1> pc1 = ...;
PCollection<T2> pc2 =
pc1.apply(ParDo.of(new MyDoFn<T1,KV<K,V>>()))
.apply(GroupByKey.<K, V>create())
.apply(Combine.perKey(new MyKeyedCombineFn<K,V>()))
.apply(ParDo.of(new MyDoFn2<KV<K,V>,T2>()));
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:
...
.apply("Step1", ParDo.of(new MyDoFn3()))
...
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 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 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 CoderRegistry.registerCoderProvider(org.apache.beam.sdk.coders.CoderProvider)
. 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(org.apache.beam.sdk.coders.Coder<T>)
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(InputT)
, computing its Output value from its InputT
value. 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()
.
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 *Fn
s 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 *Fn
s to be written conveniently, PTransform
is marked
as Serializable
, and includes dummy writeObject()
and readObject()
operations that do not save or restore any state.
Modifier and Type | Field and Description |
---|---|
protected @Nullable java.lang.String |
name
The base name of this
PTransform , e.g., from defaults, or null if not yet
assigned. |
protected @NonNull ResourceHints |
resourceHints |
Modifier | Constructor and Description |
---|---|
protected |
PTransform() |
protected |
PTransform(@Nullable java.lang.String name) |
Modifier and Type | Method and Description |
---|---|
static <InputT extends PInput,OutputT extends POutput> |
compose(SerializableFunction<InputT,OutputT> fn)
For a
SerializableFunction<InputT, OutputT> fn , returns a PTransform
given by applying fn.apply(v) to the input PCollection<InputT> . |
static <InputT extends PInput,OutputT extends POutput> |
compose(java.lang.String name,
SerializableFunction<InputT,OutputT> fn)
Like
compose(SerializableFunction) , but with a custom name. |
abstract OutputT |
expand(InputT input)
Override this method to specify how this
PTransform should be expanded on the given
InputT . |
java.util.Map<TupleTag<?>,PValue> |
getAdditionalInputs()
Returns all
PValues that are consumed as inputs to this PTransform that
are independent of the expansion of the InputT within expand(PInput) . |
protected Coder<?> |
getDefaultOutputCoder()
Deprecated.
Instead, the PTransform should explicitly call
PCollection.setCoder(org.apache.beam.sdk.coders.Coder<T>) on the
returned PCollection. |
protected Coder<?> |
getDefaultOutputCoder(InputT input)
Deprecated.
Instead, the PTransform should explicitly call
PCollection.setCoder(org.apache.beam.sdk.coders.Coder<T>) on the
returned PCollection. |
<T> Coder<T> |
getDefaultOutputCoder(InputT input,
PCollection<T> output)
Deprecated.
Instead, the PTransform should explicitly call
PCollection.setCoder(org.apache.beam.sdk.coders.Coder<T>) on the
returned PCollection. |
protected java.lang.String |
getKindString()
Returns the name to use by default for this
PTransform (not including the names of any
enclosing PTransform s). |
java.lang.String |
getName()
Returns the transform name.
|
ResourceHints |
getResourceHints()
Returns resource hints set on the transform.
|
void |
populateDisplayData(DisplayData.Builder builder)
Register display data for the given transform or component.
|
PTransform<InputT,OutputT> |
setResourceHints(@NonNull ResourceHints resourceHints)
Sets resource hints for the transform.
|
java.lang.String |
toString() |
void |
validate(@Nullable PipelineOptions options)
Called before running the Pipeline to verify this transform is fully and correctly specified.
|
protected final transient @Nullable java.lang.String name
PTransform
, e.g., from defaults, or null
if not yet
assigned.protected transient @NonNull ResourceHints resourceHints
protected PTransform()
protected PTransform(@Nullable java.lang.String name)
public abstract OutputT expand(InputT input)
PTransform
should be expanded on the given
InputT
.
NOTE: This method should not be called directly. Instead apply the PTransform
should
be applied to the InputT
using the apply
method.
Composite transforms, which are defined in terms of other transforms, should return the output of one of the composed transforms. Non-composite transforms, which do not apply any transforms internally, should return a new unbound output and register evaluators (via backend-specific registration methods).
public void validate(@Nullable PipelineOptions options)
By default, does nothing.
public java.util.Map<TupleTag<?>,PValue> getAdditionalInputs()
PValues
that are consumed as inputs to this PTransform
that
are independent of the expansion of the InputT
within expand(PInput)
.
For example, this can contain any side input consumed by this PTransform
.
public java.lang.String getName()
This name is provided by the transform creator and is not required to be unique.
public PTransform<InputT,OutputT> setResourceHints(@NonNull ResourceHints resourceHints)
resourceHints
- a ResourceHints
instance.For example:
Pipeline p = ...
...
p.apply(new SomeTransform().setResourceHints(ResourceHints.create().withMinRam("6 GiB")))
...
public ResourceHints getResourceHints()
public java.lang.String toString()
toString
in class java.lang.Object
protected java.lang.String getKindString()
PTransform
(not including the names of any
enclosing PTransform
s).
By default, returns the base name of this PTransform
's class.
The caller is responsible for ensuring that names of applied PTransform
s are unique,
e.g., by adding a uniquifying suffix when needed.
@Deprecated protected Coder<?> getDefaultOutputCoder() throws CannotProvideCoderException
PCollection.setCoder(org.apache.beam.sdk.coders.Coder<T>)
on the
returned PCollection.Coder
to use for the output of this single-output PTransform
.
By default, always throws
CannotProvideCoderException
- if no coder can be inferred@Deprecated protected Coder<?> getDefaultOutputCoder(InputT input) throws CannotProvideCoderException
PCollection.setCoder(org.apache.beam.sdk.coders.Coder<T>)
on the
returned PCollection.Coder
to use for the output of this single-output PTransform
when applied to the given input.
By default, always throws.
CannotProvideCoderException
- if none can be inferred.@Deprecated public <T> Coder<T> getDefaultOutputCoder(InputT input, PCollection<T> output) throws CannotProvideCoderException
PCollection.setCoder(org.apache.beam.sdk.coders.Coder<T>)
on the
returned PCollection.Coder
to use for the given output of this single-output PTransform
when applied to the given input.
By default, always throws.
CannotProvideCoderException
- if none can be inferred.public void populateDisplayData(DisplayData.Builder builder)
populateDisplayData(DisplayData.Builder)
is invoked by Pipeline runners to collect
display data via DisplayData.from(HasDisplayData)
. Implementations may call super.populateDisplayData(builder)
in order to register display data in the current namespace,
but should otherwise use subcomponent.populateDisplayData(builder)
to use the namespace
of the subcomponent.
By default, does not register any display data. Implementors may override this method to provide their own display data.
populateDisplayData
in interface HasDisplayData
builder
- The builder to populate with display data.HasDisplayData
@Experimental public static <InputT extends PInput,OutputT extends POutput> PTransform<InputT,OutputT> compose(SerializableFunction<InputT,OutputT> fn)
SerializableFunction<InputT, OutputT>
fn
, returns a PTransform
given by applying fn.apply(v)
to the input PCollection<InputT>
.
Allows users to define a concise composite transform using a Java 8 lambda expression. For example:
PCollection<String> words = wordsAndErrors.apply(
(PCollectionTuple input) -> {
input.get(errorsTag).apply(new WriteErrorOutput());
return input.get(wordsTag);
});
@Experimental public static <InputT extends PInput,OutputT extends POutput> PTransform<InputT,OutputT> compose(java.lang.String name, SerializableFunction<InputT,OutputT> fn)
compose(SerializableFunction)
, but with a custom name.