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 *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.
| Modifier and Type | Field and Description | 
|---|---|
| protected java.lang.String | nameThe base name of this  PTransform, e.g., from defaults, ornullif not yet
 assigned. | 
| Modifier | Constructor and Description | 
|---|---|
| protected  | PTransform() | 
| protected  | PTransform(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 aPTransformgiven by applyingfn.apply(v)to the inputPCollection<InputT>. | 
| abstract OutputT | expand(InputT input)Override this method to specify how this  PTransformshould be expanded on the givenInputT. | 
| java.util.Map<TupleTag<?>,PValue> | getAdditionalInputs()Returns all  PValuesthat are consumed as inputs to thisPTransformthat
 are independent of the expansion of theInputTwithinexpand(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
 enclosingPTransforms). | 
| java.lang.String | getName()Returns the transform name. | 
| void | populateDisplayData(DisplayData.Builder builder)Register display data for the given transform or component. | 
| java.lang.String | toString() | 
| void | validate(PipelineOptions options)Called before running the Pipeline to verify this transform is fully and correctly specified. | 
@Nullable protected final transient java.lang.String name
PTransform, e.g., from defaults, or null if not yet
 assigned.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 java.lang.String toString()
toString in class java.lang.Objectprotected java.lang.String getKindString()
PTransform (not including the names of any
 enclosing PTransforms).
 By default, returns the base name of this PTransform's class.
 
The caller is responsible for ensuring that names of applied PTransforms 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 HasDisplayDatabuilder - 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);
   });