Class PTransform<InputT extends PInput,OutputT extends POutput>
- java.lang.Object
-
- org.apache.beam.sdk.transforms.PTransform<InputT,OutputT>
-
- Type Parameters:
InputT
- the type of the input to this PTransformOutputT
- the type of the output of this PTransform
- All Implemented Interfaces:
java.io.Serializable
,HasDisplayData
- Direct Known Subclasses:
AddFields.Inner
,ApproximateUnique.Globally
,ApproximateUnique.PerKey
,AvroIO.Parse
,AvroIO.ParseAll
,AvroIO.ParseFiles
,AvroIO.Read
,AvroIO.ReadAll
,AvroIO.ReadFiles
,AvroIO.TypedWrite
,AvroIO.Write
,BoundedReadFromUnboundedSource
,Cast
,CoGroup.ExpandCrossProduct
,CoGroup.Impl
,CoGroupByKey
,Combine.Globally
,Combine.GloballyAsSingletonView
,Combine.GroupedValues
,Combine.PerKey
,Combine.PerKeyWithHotKeyFanout
,Create.OfValueProvider
,Create.TimestampedValues
,Create.Values
,DeadLetteredTransform
,Deduplicate.KeyedValues
,Deduplicate.Values
,Deduplicate.WithRepresentativeValues
,Distinct
,Distinct.WithRepresentativeValues
,DropFields.Inner
,FileIO.Match
,FileIO.MatchAll
,FileIO.ReadMatches
,FileIO.Write
,Filter
,Filter.Inner
,FlatMapElements
,FlatMapElements.FlatMapWithFailures
,Flatten.Iterables
,Flatten.PCollections
,GenerateSequence
,Group.AggregateCombiner
,Group.CombineGlobally
,Group.Global
,GroupByKey
,GroupIntoBatches
,GroupIntoBatches.WithShardedKey
,Impulse
,Join.Impl
,JsonToRow.JsonToRowWithErrFn
,Keys
,KvSwap
,MapElements
,MapElements.MapWithFailures
,MapKeys
,MapValues
,ParDo.MultiOutput
,ParDo.SingleOutput
,Partition
,PAssert.DefaultConcludeTransform
,PAssert.GroupThenAssert
,PAssert.GroupThenAssertForSingleton
,PAssert.OneSideInputAssert
,PeriodicImpulse
,PeriodicSequence
,Read.Bounded
,Read.Unbounded
,ReadAllViaFileBasedSource
,Regex.AllMatches
,Regex.Find
,Regex.FindAll
,Regex.FindKV
,Regex.FindName
,Regex.FindNameKV
,Regex.Matches
,Regex.MatchesKV
,Regex.MatchesName
,Regex.MatchesNameKV
,Regex.ReplaceAll
,Regex.ReplaceFirst
,Regex.Split
,RenameFields.Inner
,Reshuffle
,Reshuffle.ViaRandomKey
,Select.Fields
,Select.Flattened
,TestStream
,TextIO.Read
,TextIO.ReadAll
,TextIO.ReadFiles
,TextIO.TypedWrite
,TextIO.Write
,TFRecordIO.Read
,TFRecordIO.ReadFiles
,TFRecordIO.Write
,ToJson
,Values
,View.AsIterable
,View.AsList
,View.AsMap
,View.AsMultimap
,View.AsSingleton
,View.CreatePCollectionView
,Wait.OnSignal
,Watch.Growth
,Window
,Window.Assign
,WithKeys
,WithKeys
,WithTimestamps
,WriteFiles
public abstract class PTransform<InputT extends PInput,OutputT extends POutput> extends java.lang.Object implements java.io.Serializable, HasDisplayData
APTransform<InputT, OutputT>
is an operation that takes anInputT
(some subtype ofPInput
) and produces anOutputT
(some subtype ofPOutput
).Common PTransforms include root PTransforms like
TextIO.Read
,Create
, processing and conversion operations likeParDo
,GroupByKey
,CoGroupByKey
,Combine
, andCount
, and outputting PTransforms likeTextIO.Write
. Users also define their own application-specific composite PTransforms.Each
PTransform<InputT, OutputT>
has a singleInputT
type and a singleOutputT
type. Many PTransforms conceptually transform one input value to one output value, and in this caseInputT
andOutput
are typically instances ofPCollection
. A root PTransform conceptually has no input; in this case, conventionally aPBegin
object produced by callingPipeline.begin()
is used as the input. An outputting PTransform conceptually has no output; in this case, conventionallyPDone
is used as its output type. Some PTransforms conceptually have multiple inputs and/or outputs; in these cases special "bundling" classes likePCollectionList
,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 callingapply()
on itsInputT
, returning itsOutputT
. 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 enclosingPipeline
'sCoderRegistry
(accessible viaPipeline.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, viaCoderRegistry.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 callingPCollection.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 itsInputT
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, usingapply()
.Note on Serialization
PTransform
doesn't actually support serialization, despite implementingSerializable
.PTransform
is markedSerializable
solely because it is common for an anonymousDoFn
, instance to be created within anapply()
method of a compositePTransform
.Each of those
*Fn
s isSerializable
, but unfortunately its instance state will contain a reference to the enclosingPTransform
instance, and so attempt to serialize thePTransform
instance, even though the*Fn
instance never references anything about the enclosingPTransform
.To allow such anonymous
*Fn
s to be written conveniently,PTransform
is marked asSerializable
, and includes dummywriteObject()
andreadObject()
operations that do not save or restore any state.- See Also:
- Applying Transformations, Serialized Form
-
-
Field Summary
Fields Modifier and Type Field Description protected @Nullable java.lang.String
name
The base name of thisPTransform
, e.g., from defaults, ornull
if not yet assigned.protected @NonNull ResourceHints
resourceHints
-
Constructor Summary
Constructors Modifier Constructor Description protected
PTransform()
protected
PTransform(@Nullable java.lang.String name)
-
Method Summary
All Methods Static Methods Instance Methods Abstract Methods Concrete Methods Deprecated Methods Modifier and Type Method Description static <InputT extends PInput,OutputT extends POutput>
PTransform<InputT,OutputT>compose(java.lang.String name, SerializableFunction<InputT,OutputT> fn)
Likecompose(SerializableFunction)
, but with a custom name.static <InputT extends PInput,OutputT extends POutput>
PTransform<InputT,OutputT>compose(SerializableFunction<InputT,OutputT> fn)
For aSerializableFunction<InputT, OutputT>
fn
, returns aPTransform
given by applyingfn.apply(v)
to the inputPCollection<InputT>
.abstract OutputT
expand(InputT input)
Override this method to specify how thisPTransform
should be expanded on the givenInputT
.java.util.Map<TupleTag<?>,PValue>
getAdditionalInputs()
Returns allPValues
that are consumed as inputs to thisPTransform
that are independent of the expansion of thePTransform
withinexpand(PInput)
.protected Coder<?>
getDefaultOutputCoder()
Deprecated.Instead, the PTransform should explicitly callPCollection.setCoder(org.apache.beam.sdk.coders.Coder<T>)
on the returned PCollection.protected Coder<?>
getDefaultOutputCoder(InputT input)
Deprecated.Instead, the PTransform should explicitly callPCollection.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 callPCollection.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 thisPTransform
(not including the names of any enclosingPTransform
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.void
validate(@Nullable PipelineOptions options, java.util.Map<TupleTag<?>,PCollection<?>> inputs, java.util.Map<TupleTag<?>,PCollection<?>> outputs)
Called before running the Pipeline to verify this transform, its inputs, and outputs are fully and correctly specified.
-
-
-
Field Detail
-
name
protected final transient @Nullable java.lang.String name
The base name of thisPTransform
, e.g., from defaults, ornull
if not yet assigned.
-
resourceHints
protected transient @NonNull ResourceHints resourceHints
-
-
Method Detail
-
expand
public abstract OutputT expand(InputT input)
Override this method to specify how thisPTransform
should be expanded on the givenInputT
.NOTE: This method should not be called directly. Instead apply the
PTransform
should be applied to theInputT
using theapply
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).
-
validate
public void validate(@Nullable PipelineOptions options)
Called before running the Pipeline to verify this transform is fully and correctly specified.By default, does nothing.
-
validate
public void validate(@Nullable PipelineOptions options, java.util.Map<TupleTag<?>,PCollection<?>> inputs, java.util.Map<TupleTag<?>,PCollection<?>> outputs)
Called before running the Pipeline to verify this transform, its inputs, and outputs are fully and correctly specified.By default, delegates to
validate(PipelineOptions)
.
-
getAdditionalInputs
public java.util.Map<TupleTag<?>,PValue> getAdditionalInputs()
Returns allPValues
that are consumed as inputs to thisPTransform
that are independent of the expansion of thePTransform
withinexpand(PInput)
.For example, this can contain any side input consumed by this
PTransform
.
-
getName
public java.lang.String getName()
Returns the transform name.This name is provided by the transform creator and is not required to be unique.
-
setResourceHints
public PTransform<InputT,OutputT> setResourceHints(@NonNull ResourceHints resourceHints)
Sets resource hints for the transform.- Parameters:
resourceHints
- aResourceHints
instance.- Returns:
- a reference to the same transfrom instance.
For example:
Pipeline p = ... ... p.apply(new SomeTransform().setResourceHints(ResourceHints.create().withMinRam("6 GiB"))) ...
-
getResourceHints
public ResourceHints getResourceHints()
Returns resource hints set on the transform.
-
toString
public java.lang.String toString()
- Overrides:
toString
in classjava.lang.Object
-
getKindString
protected java.lang.String getKindString()
Returns the name to use by default for thisPTransform
(not including the names of any enclosingPTransform
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.
-
getDefaultOutputCoder
@Deprecated protected Coder<?> getDefaultOutputCoder() throws CannotProvideCoderException
Deprecated.Instead, the PTransform should explicitly callPCollection.setCoder(org.apache.beam.sdk.coders.Coder<T>)
on the returned PCollection.Returns the defaultCoder
to use for the output of this single-outputPTransform
.By default, always throws
- Throws:
CannotProvideCoderException
- if no coder can be inferred
-
getDefaultOutputCoder
@Deprecated protected Coder<?> getDefaultOutputCoder(InputT input) throws CannotProvideCoderException
Deprecated.Instead, the PTransform should explicitly callPCollection.setCoder(org.apache.beam.sdk.coders.Coder<T>)
on the returned PCollection.Returns the defaultCoder
to use for the output of this single-outputPTransform
when applied to the given input.By default, always throws.
- Throws:
CannotProvideCoderException
- if none can be inferred.
-
getDefaultOutputCoder
@Deprecated public <T> Coder<T> getDefaultOutputCoder(InputT input, PCollection<T> output) throws CannotProvideCoderException
Deprecated.Instead, the PTransform should explicitly callPCollection.setCoder(org.apache.beam.sdk.coders.Coder<T>)
on the returned PCollection.Returns the defaultCoder
to use for the given output of this single-outputPTransform
when applied to the given input.By default, always throws.
- Throws:
CannotProvideCoderException
- if none can be inferred.
-
populateDisplayData
public void populateDisplayData(DisplayData.Builder builder)
Register display data for the given transform or component.populateDisplayData(DisplayData.Builder)
is invoked by Pipeline runners to collect display data viaDisplayData.from(HasDisplayData)
. Implementations may callsuper.populateDisplayData(builder)
in order to register display data in the current namespace, but should otherwise usesubcomponent.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.
- Specified by:
populateDisplayData
in interfaceHasDisplayData
- Parameters:
builder
- The builder to populate with display data.- See Also:
HasDisplayData
-
compose
@Experimental public static <InputT extends PInput,OutputT extends POutput> PTransform<InputT,OutputT> compose(SerializableFunction<InputT,OutputT> fn)
For aSerializableFunction<InputT, OutputT>
fn
, returns aPTransform
given by applyingfn.apply(v)
to the inputPCollection<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); });
-
compose
@Experimental public static <InputT extends PInput,OutputT extends POutput> PTransform<InputT,OutputT> compose(java.lang.String name, SerializableFunction<InputT,OutputT> fn)
Likecompose(SerializableFunction)
, but with a custom name.
-
-