public class ParDo extends Object
ParDo is the core element-wise transform in Google Cloud
Dataflow, invoking a user-specified function (from I to
Output) on each of the elements of the input
PCollection<InputT> to produce zero or more output elements, all
of which are collected into the output PCollection<OutputT>.
Elements are processed independently, and possibly in parallel across distributed cloud resources.
The ParDo processing style is similar to what happens inside
the "Mapper" or "Reducer" class of a MapReduce-style algorithm.
DoFnsThe function to use to process each element is specified by a
DoFn.
Conceptually, when a ParDo transform is executed, the
elements of the input PCollection<InputT> are first divided up
into some number of "batches". These are farmed off to distributed
worker machines (or run locally, if using the
DirectPipelineRunner).
For each batch of input elements, a fresh instance of the argument
DoFn<InputT, OutputT> is created on a worker, then the DoFn's
optional DoFn.startBundle(com.google.cloud.dataflow.sdk.transforms.DoFn<InputT, OutputT>.Context) method is called to initialize it,
then the DoFn's required DoFn.processElement(com.google.cloud.dataflow.sdk.transforms.DoFn<InputT, OutputT>.ProcessContext) method
is called on each of the input elements in the batch, then the
DoFn's optional DoFn.finishBundle(com.google.cloud.dataflow.sdk.transforms.DoFn<InputT, OutputT>.Context) method is called
to complete its work, and finally the DoFn instance is
thrown away. Each of the calls to any of the DoFn's
methods can produce zero or more output elements, which are
collected together into a batch of output elements. All of the
batches of output elements from all of the DoFn instances
are "flattened" together into the output PCollection<OutputT>.
For example:
PCollection<String> lines = ...;
PCollection<String> words =
lines.apply(ParDo.of(new DoFn<String, String>() {
public void processElement(ProcessContext c) {
String line = c.element();
for (String word : line.split("[^a-zA-Z']+")) {
c.output(word);
}
}}));
PCollection<Integer> wordLengths =
words.apply(ParDo.of(new DoFn<String, Integer>() {
public void processElement(ProcessContext c) {
String word = c.element();
Integer length = word.length();
c.output(length);
}}));
Each output element has the same timestamp and is in the same windows
as its corresponding input element, and the output PCollection<OutputT>
has the same
WindowFn
associated with it as the input.
ParDosA ParDo transform can be given a name using
named(java.lang.String). While the system will automatically provide a name
if none is specified explicitly, it is still a good practice to
provide an explicit name, since that will probably make monitoring
output more readable. For example:
PCollection<String> words =
lines.apply(ParDo.named("ExtractWords")
.of(new DoFn<String, String>() { ... }));
PCollection<Integer> wordLengths =
words.apply(ParDo.named("ComputeWordLengths")
.of(new DoFn<String, Integer>() { ... }));
While a ParDo iterates over a single "main input"
PCollection, it can take additional "side input"
PCollectionViews. These side input
PCollectionViews express styles of accessing
PCollections computed by earlier pipeline operations,
passed in to the ParDo transform using
withSideInputs(com.google.cloud.dataflow.sdk.values.PCollectionView<?>...), and their contents accessible to each of
the DoFn operations via sideInput.
For example:
PCollection<String> words = ...;
PCollection<Integer> maxWordLengthCutOff = ...; // Singleton PCollection
final PCollectionView<Integer> maxWordLengthCutOffView =
maxWordLengthCutOff.apply(View.<Integer>asSingleton());
PCollection<String> wordsBelowCutOff =
words.apply(ParDo.withSideInput(maxWordLengthCutOffView)
.of(new DoFn<String, String>() {
public void processElement(ProcessContext c) {
String word = c.element();
int lengthCutOff = c.sideInput(maxWordLengthCutOffView);
if (word.length() <= lengthCutOff) {
c.output(word);
}
}}));
Optionally, a ParDo transform can produce multiple
output PCollections, both a "main output"
PCollection<OutputT> plus any number of "side output"
PCollections, each keyed by a distinct TupleTag,
and bundled in a PCollectionTuple. The TupleTags
to be used for the output PCollectionTuple is specified by
invoking withOutputTags(com.google.cloud.dataflow.sdk.values.TupleTag<OutputT>, com.google.cloud.dataflow.sdk.values.TupleTagList). Unconsumed side outputs does not
necessarily need to be explicity specified, even if the DoFn
generates them. Within the DoFn, an element is added to the
main output PCollection as normal, using
DoFn.Context.output(OutputT), while an element is added to a side output
PCollection using DoFn.Context.sideOutput(com.google.cloud.dataflow.sdk.values.TupleTag<T>, T). For example:
PCollection<String> words = ...;
// Select words whose length is below a cut off,
// plus the lengths of words that are above the cut off.
// Also select words starting with "MARKER".
final int wordLengthCutOff = 10;
// Create tags to use for the main and side outputs.
final TupleTag<String> wordsBelowCutOffTag =
new TupleTag<String>(){};
final TupleTag<Integer> wordLengthsAboveCutOffTag =
new TupleTag<Integer>(){};
final TupleTag<String> markedWordsTag =
new TupleTag<String>(){};
PCollectionTuple results =
words.apply(
ParDo
// Specify the main and consumed side output tags of the
// PCollectionTuple result:
.withOutputTags(wordsBelowCutOffTag,
TupleTagList.of(wordLengthsAboveCutOffTag)
.and(markedWordsTag))
.of(new DoFn<String, String>() {
// Create a tag for the unconsumed side output.
final TupleTag<String> specialWordsTag =
new TupleTag<String>(){};
public void processElement(ProcessContext c) {
String word = c.element();
if (word.length() <= wordLengthCutOff) {
// Emit this short word to the main output.
c.output(word);
} else {
// Emit this long word's length to a side output.
c.sideOutput(wordLengthsAboveCutOffTag, word.length());
}
if (word.startsWith("MARKER")) {
// Emit this word to a different side output.
c.sideOutput(markedWordsTag, word);
}
if (word.startsWith("SPECIAL")) {
// Emit this word to the unconsumed side output.
c.sideOutput(specialWordsTag, word);
}
}}));
// Extract the PCollection results, by tag.
PCollection<String> wordsBelowCutOff =
results.get(wordsBelowCutOffTag);
PCollection<Integer> wordLengthsAboveCutOff =
results.get(wordLengthsAboveCutOffTag);
PCollection<String> markedWords =
results.get(markedWordsTag);
Several properties can be specified for a ParDo
PTransform, including name, side inputs, side output tags,
and DoFn to invoke. Only the DoFn is required; the
name is encouraged but not required, and side inputs and side
output tags are only specified when they're needed. These
properties can be specified in any order, as long as they're
specified before the ParDo PTransform is applied.
The approach used to allow these properties to be specified in
any order, with some properties omitted, is to have each of the
property "setter" methods defined as static factory methods on
ParDo itself, which return an instance of either
ParDo.Unbound or
ParDo.Bound nested classes, each of which offer
property setter instance methods to enable setting additional
properties. ParDo.Bound is used for ParDo
transforms whose DoFn is specified and whose input and
output static types have been bound. ParDo.Unbound is used
for ParDo transforms that have not yet had their
DoFn specified. Only ParDo.Bound instances can be
applied.
Another benefit of this approach is that it reduces the number
of type parameters that need to be specified manually. In
particular, the input and output types of the ParDo
PTransform are inferred automatically from the type
parameters of the DoFn argument passed to of(com.google.cloud.dataflow.sdk.transforms.DoFn<InputT, OutputT>).
By default, the Coder<OutputT> of the
elements of the main output PCollection<OutputT> is inferred from the
concrete type of the DoFn<InputT, OutputT>'s output type Output.
By default, the Coder<X> of the elements of a side output
PCollection<X> is inferred from the concrete type of the
corresponding TupleTag<X>'s type X. To be
successful, the TupleTag should be created as an instance
of a trivial anonymous subclass, with {} suffixed to the
constructor call. Such uses block Java's generic type parameter
inference, so the <X> argument must be provided explicitly.
For example:
// A TupleTag to use for a side input can be written concisely:
final TupleTag<Integer> sideInputag = new TupleTag<>();
// A TupleTag to use for a side output should be written with "{}",
// and explicit generic parameter type:
final TupleTag<String> sideOutputTag = new TupleTag<String>(){};
This style of TupleTag instantiation is used in the example of
multiple side outputs, above.
DoFnsA DoFn passed to a ParDo transform must be
Serializable. This allows the DoFn instance
created in this "main program" to be sent (in serialized form) to
remote worker machines and reconstituted for each batch of elements
of the input PCollection being processed. A DoFn
can have instance variable state, and non-transient instance
variable state will be serialized in the main program and then
deserialized on remote worker machines for each batch of elements
to process.
To aid in ensuring that DoFns are properly
Serializable, even local execution using the
DirectPipelineRunner will serialize and then deserialize
DoFns before executing them on a batch.
DoFns expressed as anonymous inner classes can be
convenient, but due to a quirk in Java's rules for serializability,
non-static inner or nested classes (including anonymous inner
classes) automatically capture their enclosing class's instance in
their serialized state. This can lead to including much more than
intended in the serialized state of a DoFn, or even things
that aren't Serializable.
There are two ways to avoid unintended serialized state in a
DoFn:
DoFn as a named, static class.
DoFn as an anonymous inner class inside of
a static method.
Both these approaches ensure that there is no implicit enclosing
class instance serialized along with the DoFn instance.
Prior to Java 8, any local variables of the enclosing
method referenced from within an anonymous inner class need to be
marked as final. If defining the DoFn as a named
static class, such variables would be passed as explicit
constructor arguments and stored in explicit instance variables.
There are three main ways to initialize the state of a
DoFn instance processing a batch:
DoFn's constructor (which is
implicit for an anonymous inner class). This state will be
automatically serialized and then deserialized in the DoFn
instance created for each batch. This method is good for state
known when the original DoFn is created in the main
program, if it's not overly large.
PCollection and pass it
in as a side input to the DoFn. This is good if the state
needs to be computed by the pipeline, or if the state is very large
and so is best read from file(s) rather than sent as part of the
DoFn's serialized state.
DoFn instance, in
DoFn.startBundle(com.google.cloud.dataflow.sdk.transforms.DoFn<InputT, OutputT>.Context). This is good if the initialization
doesn't depend on any information known only by the main program or
computed by earlier pipeline operations, but is the same for all
instances of this DoFn for all program executions, say
setting up empty caches or initializing constant data.
ParDo operations are intended to be able to run in
parallel across multiple worker machines. This precludes easy
sharing and updating mutable state across those machines. There is
no support in the Google Cloud Dataflow system for communicating
and synchronizing updates to shared state across worker machines,
so programs should not access any mutable static variable state in
their DoFn, without understanding that the Java processes
for the main program and workers will each have its own independent
copy of such state, and there won't be any automatic copying of
that state across Java processes. All information should be
communicated to DoFn instances via main and side inputs and
serialized state, and all output should be communicated from a
DoFn instance via main and side outputs, in the absence of
external communication mechanisms written by user code.
In a distributed system, things can fail: machines can crash,
machines can be unable to communicate across the network, etc.
While individual failures are rare, the larger the job, the greater
the chance that something, somewhere, will fail. The Google Cloud
Dataflow service strives to mask such failures automatically,
principally by retrying failed DoFn batches. This means
that a DoFn instance might process a batch partially, then
crash for some reason, then be rerun (often on a different worker
machine) on that same batch and on the same elements as before.
Sometimes two or more DoFn instances will be running on the
same batch simultaneously, with the system taking the results of
the first instance to complete successfully. Consequently, the
code in a DoFn needs to be written such that these
duplicate (sequential or concurrent) executions do not cause
problems. If the outputs of a DoFn are a pure function of
its inputs, then this requirement is satisfied. However, if a
DoFn's execution has external side-effects, say performing
updates to external HTTP services, then the DoFn's code
needs to take care to ensure that those updates are idempotent and
that concurrent updates are acceptable. This property can be
difficult to achieve, so it is advisable to strive to keep
DoFns as pure functions as much as possible.
The Google Cloud Dataflow service automatically optimizes a pipeline before it is executed. A key optimization, fusion, relates to ParDo operations. If one ParDo operation produces a PCollection that is then consumed as the main input of another ParDo operation, the two ParDo operations will be fused together into a single ParDo operation and run in a single pass; this is "producer-consumer fusion". Similarly, if two or more ParDo operations have the same PCollection main input, they will be fused into a single ParDo that makes just one pass over the input PCollection; this is "sibling fusion".
If after fusion there are no more unfused references to a PCollection (e.g., one between a producer ParDo and a consumer ParDo), the PCollection itself is "fused away" and won't ever be written to disk, saving all the I/O and space expense of constructing it.
The Google Cloud Dataflow service applies fusion as much as possible, greatly reducing the cost of executing pipelines. As a result, it is essentially "free" to write ParDo operations in a very modular, composable style, each ParDo operation doing one clear task, and stringing together sequences of ParDo operations to get the desired overall effect. Such programs can be easier to understand, easier to unit-test, easier to extend and evolve, and easier to reuse in new programs. The predefined library of PTransforms that come with Google Cloud Dataflow makes heavy use of this modular, composable style, trusting to the Google Cloud Dataflow service's optimizer to "flatten out" all the compositions into highly optimized stages.
| Modifier and Type | Class and Description |
|---|---|
static class |
ParDo.Bound<InputT,OutputT>
A
PTransform that, when applied to a PCollection<InputT>,
invokes a user-specified DoFn<InputT, OutputT> on all its elements,
with all its outputs collected into an output
PCollection<OutputT>. |
static class |
ParDo.BoundMulti<InputT,OutputT>
A
PTransform that, when applied to a
PCollection<InputT>, invokes a user-specified
DoFn<InputT, OutputT> on all its elements, which can emit elements
to any of the PTransform's main and side output
PCollections, which are bundled into a result
PCollectionTuple. |
static class |
ParDo.Unbound
An incomplete
ParDo transform, with unbound input/output types. |
static class |
ParDo.UnboundMulti<OutputT>
An incomplete multi-output
ParDo transform, with unbound
input type. |
| Constructor and Description |
|---|
ParDo() |
| Modifier and Type | Method and Description |
|---|---|
static ParDo.Unbound |
named(String name)
Creates a
ParDo PTransform with the given name. |
static <InputT,OutputT> |
of(DoFn<InputT,OutputT> fn)
|
static <InputT,OutputT> |
of(DoFnWithContext<InputT,OutputT> fn)
|
static <OutputT> ParDo.UnboundMulti<OutputT> |
withOutputTags(TupleTag<OutputT> mainOutputTag,
TupleTagList sideOutputTags)
Creates a multi-output
ParDo PTransform whose
output PCollections will be referenced using the given main
output and side output tags. |
static ParDo.Unbound |
withSideInputs(Iterable<? extends PCollectionView<?>> sideInputs)
Creates a
ParDo with the given side inputs. |
static ParDo.Unbound |
withSideInputs(PCollectionView<?>... sideInputs)
Creates a
ParDo PTransform with the given
side inputs. |
public static ParDo.Unbound named(String name)
ParDo PTransform with the given name.
See the discussion of Naming above for more explanation.
The resulting PTransform is incomplete, and its
input/output types are not yet bound. Use
ParDo.Unbound.of(com.google.cloud.dataflow.sdk.transforms.DoFn<InputT, OutputT>) to specify the DoFn to
invoke, which will also bind the input/output types of this
PTransform.
public static ParDo.Unbound withSideInputs(PCollectionView<?>... sideInputs)
ParDo PTransform with the given
side inputs.
Side inputs are PCollectionViews, whose contents are
computed during pipeline execution and then made accessible to
DoFn code via sideInput. Each
invocation of the DoFn receives the same values for these
side inputs.
See the discussion of Side Inputs above for more explanation.
The resulting PTransform is incomplete, and its
input/output types are not yet bound. Use
ParDo.Unbound.of(com.google.cloud.dataflow.sdk.transforms.DoFn<InputT, OutputT>) to specify the DoFn to
invoke, which will also bind the input/output types of this
PTransform.
public static ParDo.Unbound withSideInputs(Iterable<? extends PCollectionView<?>> sideInputs)
ParDo with the given side inputs.
Side inputs are PCollectionViews, whose contents are
computed during pipeline execution and then made accessible to
DoFn code via sideInput.
See the discussion of Side Inputs above for more explanation.
The resulting PTransform is incomplete, and its
input/output types are not yet bound. Use
ParDo.Unbound.of(com.google.cloud.dataflow.sdk.transforms.DoFn<InputT, OutputT>) to specify the DoFn to
invoke, which will also bind the input/output types of this
PTransform.
public static <OutputT> ParDo.UnboundMulti<OutputT> withOutputTags(TupleTag<OutputT> mainOutputTag, TupleTagList sideOutputTags)
ParDo PTransform whose
output PCollections will be referenced using the given main
output and side output tags.
TupleTags are used to name (with its static element
type T) each main and side output PCollection<T>.
This PTransform's DoFn emits elements to the main
output PCollection as normal, using
DoFn.Context.output(OutputT). The DoFn emits elements to
a side output PCollection using
DoFn.Context.sideOutput(com.google.cloud.dataflow.sdk.values.TupleTag<T>, T), passing that side output's tag
as an argument. The result of invoking this PTransform
will be a PCollectionTuple, and any of the the main and
side output PCollections can be retrieved from it via
PCollectionTuple.get(com.google.cloud.dataflow.sdk.values.TupleTag<T>), passing the output's tag as an
argument.
See the discussion of Side Outputs above for more explanation.
The resulting PTransform is incomplete, and its input
type is not yet bound. Use ParDo.UnboundMulti.of(com.google.cloud.dataflow.sdk.transforms.DoFn<InputT, OutputT>)
to specify the DoFn to invoke, which will also bind the
input type of this PTransform.
public static <InputT,OutputT> ParDo.Bound<InputT,OutputT> of(DoFn<InputT,OutputT> fn)
ParDo PTransform that will invoke the
given DoFn function.
The resulting PTransform's types have been bound, with the
input being a PCollection<InputT> and the output a
PCollection<OutputT>, inferred from the types of the argument
DoFn<InputT, OutputT>. It is ready to be applied, or further
properties can be set on it first.
@Experimental public static <InputT,OutputT> ParDo.Bound<InputT,OutputT> of(DoFnWithContext<InputT,OutputT> fn)
ParDo PTransform that will invoke the
given DoFnWithContext function.
The resulting PTransform's types have been bound, with the
input being a PCollection<InputT> and the output a
PCollection<OutputT>, inferred from the types of the argument
DoFn<InputT, OutputT>. It is ready to be applied, or further
properties can be set on it first.
DoFnWithContext is an experimental alternative to
DoFn which simplifies accessing the window of the element.