Class Flatten
Flatten<T> takes multiple PCollection<T>s bundled into a 
 PCollectionList<T> and returns a single PCollection<T> containing all the elements in
 all the input PCollections. The name "Flatten" suggests taking a list of lists and
 flattening them into a single list.
 Example of use:
 PCollection<String> pc1 = ...;
 PCollection<String> pc2 = ...;
 PCollection<String> pc3 = ...;
 PCollectionList<String> pcs = PCollectionList.of(pc1).and(pc2).and(pc3);
 PCollection<String> merged = pcs.apply(Flatten.<String>pCollections());
 
 By default, the Coder of the output PCollection is the same as the 
 Coder of the first PCollection in the input PCollectionList (if the 
 PCollectionList is non-empty).
- 
Nested Class Summary
Nested ClassesModifier and TypeClassDescriptionstatic classFlattenIterables<T>takes aPCollection<Iterable<T>>and returns aPCollection<T>that contains all the elements from each iterable.static classAPTransformthat flattens aPCollectionListinto aPCollectioncontaining all the elements of all thePCollections in its input. - 
Constructor Summary
Constructors - 
Method Summary
Modifier and TypeMethodDescriptionstatic <T> Flatten.Iterables<T> Returns aPTransformthat takes aPCollection<Iterable<T>>and returns aPCollection<T>containing all the elements from all theIterables.static <T> Flatten.PCollections<T> Returns aPTransformthat flattens aPCollectionListinto aPCollectioncontaining all the elements of all thePCollections in its input.static <T> PTransform<PCollection<T>, PCollection<T>> with(PTransform<PBegin, PCollection<T>> other) Returns aPTransformthat flattens the inputPCollectionwith the output of anotherPTransformresulting in aPCollectioncontaining all the elements of both the inputPCollections and the output of the givenPTransformas its output.static <T> PTransform<PCollection<T>, PCollection<T>> with(PCollection<T> other) Returns aPTransformthat flattens the inputPCollectionwith a givenPCollectionresulting in aPCollectioncontaining all the elements of bothPCollections as its output. 
- 
Constructor Details
- 
Flatten
public Flatten() 
 - 
 - 
Method Details
- 
pCollections
Returns aPTransformthat flattens aPCollectionListinto aPCollectioncontaining all the elements of all thePCollections in its input.All inputs must have equal
WindowFns. The output elements ofFlatten<T>are in the same windows and have the same timestamps as their corresponding input elements. The outputPCollectionwill have the sameWindowFnas all of the inputs.- Type Parameters:
 T- the type of the elements in the input and outputPCollections.
 - 
iterables
Returns aPTransformthat takes aPCollection<Iterable<T>>and returns aPCollection<T>containing all the elements from all theIterables.Example of use:
PCollection<Iterable<Integer>> pcOfIterables = ...; PCollection<Integer> pc = pcOfIterables.apply(Flatten.<Integer>iterables());By default, the output
PCollectionencodes its elements using the sameCoderthat the input uses for the elements in itsIterable.- Type Parameters:
 T- the type of the elements of the inputIterableand the outputPCollection
 - 
with
Returns aPTransformthat flattens the inputPCollectionwith a givenPCollectionresulting in aPCollectioncontaining all the elements of bothPCollections as its output.This is equivalent to creating a
PCollectionListcontaining both the input andotherand then applyingpCollections(), but has the advantage that it can be more easily used inline.Both must have equal
WindowFns. The output elements ofFlatten<T>are in the same windows and have the same timestamps as their corresponding input elements. The outputPCollectionwill have the sameWindowFnas both inputs.- Type Parameters:
 T- the type of the elements in the input and outputPCollections.- Parameters:
 other- the other PCollection to flatten with the input
 - 
with
public static <T> PTransform<PCollection<T>,PCollection<T>> with(PTransform<PBegin, PCollection<T>> other) Returns aPTransformthat flattens the inputPCollectionwith the output of anotherPTransformresulting in aPCollectioncontaining all the elements of both the inputPCollections and the output of the givenPTransformas its output.This is equivalent to creating a
PCollectionListcontaining both the input and the output ofotherand then applyingpCollections(), but has the advantage that it can be more easily used inline.Both
PCollectionsmust have equalWindowFns. The output elements ofFlatten<T>are in the same windows and have the same timestamps as their corresponding input elements. The outputPCollectionwill have the sameWindowFnas both inputs.- Type Parameters:
 T- the type of the elements in the input and outputPCollections.- Parameters:
 other- a PTransform whose ouptput should be flattened with the input
 
 -