T - the type of the elements of this PCollectionpublic class PCollection<T> extends PValueBase implements PValue
PCollection<T> is an immutable collection of values of type T. A PCollection can contain either a bounded or unbounded number of elements. Bounded
 and unbounded PCollections are produced as the output of PTransforms (including root PTransforms like Read and Create), and can be passed
 as the inputs of other PTransforms.
 Some root transforms produce bounded PCollections and others produce unbounded ones.
 For example, GenerateSequence.from(long) with GenerateSequence.to(long) produces a fixed set
 of integers, so it produces a bounded PCollection. GenerateSequence.from(long) without
 a GenerateSequence.to(long) produces all integers as an infinite stream, so it produces an
 unbounded PCollection.
 
Each element in a PCollection has an associated timestamp. Readers assign timestamps
 to elements when they create PCollections, and other PTransforms propagate these timestamps from their input to their output. See the documentation
 on BoundedSource.BoundedReader and UnboundedSource.UnboundedReader for more information on how these readers
 produce timestamps and watermarks.
 
Additionally, a PCollection has an associated WindowFn and each element is
 assigned to a set of windows. By default, the windowing function is GlobalWindows and all
 elements are assigned into a single default window. This default can be overridden with the
 Window PTransform.
 
See the individual PTransform subclasses for specific information on how they
 propagate timestamps and windowing.
| Modifier and Type | Class and Description | 
|---|---|
| static class  | PCollection.IsBoundedThe enumeration of cases for whether a  PCollectionis bounded. | 
| Modifier and Type | Method and Description | 
|---|---|
| <OutputT extends POutput> | apply(PTransform<? super PCollection<T>,OutputT> t)of the  PTransform. | 
| <OutputT extends POutput> | apply(java.lang.String name,
     PTransform<? super PCollection<T>,OutputT> t)Applies the given  PTransformto this inputPCollection, usingnameto
 identify this specific application of the transform. | 
| static <T> PCollection<T> | createPrimitiveOutputInternal(Pipeline pipeline,
                             WindowingStrategy<?,?> windowingStrategy,
                             PCollection.IsBounded isBounded,
                             @Nullable Coder<T> coder)For internal use only; no backwards-compatibility guarantees. | 
| static <T> PCollection<T> | createPrimitiveOutputInternal(Pipeline pipeline,
                             WindowingStrategy<?,?> windowingStrategy,
                             PCollection.IsBounded isBounded,
                             @Nullable Coder<T> coder,
                             TupleTag<?> tag)For internal use only; no backwards-compatibility guarantees. | 
| java.util.Map<TupleTag<?>,PValue> | expand() | 
| void | finishSpecifying(PInput input,
                PTransform<?,?> transform)After building, finalizes this  PValueto make it ready for running. | 
| void | finishSpecifyingOutput(java.lang.String transformName,
                      PInput input,
                      PTransform<?,?> transform)As part of applying the producing  PTransform, finalizes this output to make it ready
 for being used as an input and for running. | 
| Coder<T> | getCoder()Returns the  Coderused by thisPCollectionto encode and decode the values
 stored in it. | 
| SerializableFunction<Row,T> | getFromRowFunction()Returns the attached schema's fromRowFunction. | 
| java.lang.String | getName()Returns the name of this  PCollection. | 
| Schema | getSchema()Returns the attached schema. | 
| SerializableFunction<T,Row> | getToRowFunction()Returns the attached schema's toRowFunction. | 
| @Nullable TypeDescriptor<T> | getTypeDescriptor()Returns a  TypeDescriptor<T>with some reflective information aboutT, if possible. | 
| WindowingStrategy<?,?> | getWindowingStrategy()Returns the  WindowingStrategyof thisPCollection. | 
| boolean | hasSchema()Returns whether this  PCollectionhas an attached schema. | 
| PCollection.IsBounded | isBounded() | 
| PCollection<T> | setCoder(Coder<T> coder)Sets the  Coderused by thisPCollectionto encode and decode the values stored
 in it. | 
| PCollection<T> | setIsBoundedInternal(PCollection.IsBounded isBounded)For internal use only; no backwards-compatibility guarantees. | 
| PCollection<T> | setName(java.lang.String name)Sets the name of this  PCollection. | 
| PCollection<T> | setRowSchema(Schema schema)Sets a schema on this PCollection. | 
| PCollection<T> | setSchema(Schema schema,
         TypeDescriptor<T> typeDescriptor,
         SerializableFunction<T,Row> toRowFunction,
         SerializableFunction<Row,T> fromRowFunction)Sets a  Schemaon thisPCollection. | 
| PCollection<T> | setTypeDescriptor(TypeDescriptor<T> typeDescriptor)Sets the  TypeDescriptor<T>for thisPCollection<T>. | 
| PCollection<T> | setWindowingStrategyInternal(WindowingStrategy<?,?> windowingStrategy)For internal use only; no backwards-compatibility guarantees. | 
getKindString, getPipeline, toStringclone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, waitgetPipelinegetPipelinepublic void finishSpecifyingOutput(java.lang.String transformName,
                                   PInput input,
                                   PTransform<?,?> transform)
POutputPTransform, finalizes this output to make it ready
 for being used as an input and for running.
 This includes ensuring that all PCollections have Coders specified or defaulted.
 
Automatically invoked whenever this POutput is output, after POutput.finishSpecifyingOutput(String, PInput, PTransform) has been called on each component
 PValue returned by POutput.expand().
finishSpecifyingOutput in interface POutputfinishSpecifyingOutput in class PValueBasepublic void finishSpecifying(PInput input, PTransform<?,?> transform)
PValue to make it ready for running. Automatically
 invoked whenever the PValue is "used" (e.g., when apply() is called on it) and when the
 Pipeline is run (useful if this is a PValue with no consumers).finishSpecifying in interface PValuefinishSpecifying in class PValueBaseinput - the PInput the PTransform was applied to to produce this
     outputtransform - the PTransform that produced this PValuepublic @Nullable TypeDescriptor<T> getTypeDescriptor()
TypeDescriptor<T> with some reflective information about
 T, if possible. May return null if no information is available. Subclasses may
 override this to enable better Coder inference.public java.lang.String getName()
PCollection.
 By default, the name of a PCollection is based on the name of the PTransform
 that produces it. It can be specified explicitly by calling setName(java.lang.String).
getName in interface PValuegetName in class PValueBasejava.lang.IllegalStateException - if the name hasn't been set yetpublic final java.util.Map<TupleTag<?>,PValue> expand()
PValuePOutput into a list of its component output PValues.
 PValue expands to itself.
   PValues (such as PCollectionTuple or PCollectionList) expands to its component PValue PValues.
 Not intended to be invoked directly by user code..
public PCollection<T> setName(java.lang.String name)
PCollection. Returns this.setName in class PValueBasejava.lang.IllegalStateException - if this PCollection has already been finalized and may no
     longer be set. Once apply(org.apache.beam.sdk.transforms.PTransform<? super org.apache.beam.sdk.values.PCollection<T>, OutputT>) has been called, this will be the case.public Coder<T> getCoder()
Coder used by this PCollection to encode and decode the values
 stored in it.java.lang.IllegalStateException - if the Coder hasn't been set, and couldn't be inferred.public PCollection<T> setCoder(Coder<T> coder)
java.lang.IllegalStateException - if this PCollection has already been finalized and may no
     longer be set. Once apply(org.apache.beam.sdk.transforms.PTransform<? super org.apache.beam.sdk.values.PCollection<T>, OutputT>) has been called, this will be the case.public PCollection<T> setRowSchema(Schema schema)
Can only be called on a PCollection.
public PCollection<T> setSchema(Schema schema, TypeDescriptor<T> typeDescriptor, SerializableFunction<T,Row> toRowFunction, SerializableFunction<Row,T> fromRowFunction)
Schema on this PCollection.public boolean hasSchema()
PCollection has an attached schema.public Schema getSchema()
public SerializableFunction<T,Row> getToRowFunction()
public SerializableFunction<Row,T> getFromRowFunction()
public <OutputT extends POutput> OutputT apply(PTransform<? super PCollection<T>,OutputT> t)
PTransform.PTransformpublic <OutputT extends POutput> OutputT apply(java.lang.String name, PTransform<? super PCollection<T>,OutputT> t)
PTransform to this input PCollection, using name to
 identify this specific application of the transform. This name is used in various places,
 including the monitoring UI, logging, and to stably identify this application node in the job
 graph.PTransformpublic WindowingStrategy<?,?> getWindowingStrategy()
WindowingStrategy of this PCollection.public PCollection.IsBounded isBounded()
public PCollection<T> setTypeDescriptor(TypeDescriptor<T> typeDescriptor)
TypeDescriptor<T> for this PCollection<T>. This may allow the enclosing PCollectionTuple, PCollectionList, or PTransform<?, PCollection<T>>, etc., to provide more detailed
 reflective information.@Internal public PCollection<T> setWindowingStrategyInternal(WindowingStrategy<?,?> windowingStrategy)
@Internal public PCollection<T> setIsBoundedInternal(PCollection.IsBounded isBounded)
@Internal public static <T> PCollection<T> createPrimitiveOutputInternal(Pipeline pipeline, WindowingStrategy<?,?> windowingStrategy, PCollection.IsBounded isBounded, @Nullable Coder<T> coder)
@Internal public static <T> PCollection<T> createPrimitiveOutputInternal(Pipeline pipeline, WindowingStrategy<?,?> windowingStrategy, PCollection.IsBounded isBounded, @Nullable Coder<T> coder, TupleTag<?> tag)