ParDo (Google Cloud Dataflow SDK 1.9.1 API)

Google Cloud Dataflow SDK for Java, version 1.9.1

com.google.cloud.dataflow.sdk.transforms

Class ParDo



  • public class ParDo
    extends Object
    ParDo is the core element-wise transform in Google Cloud Dataflow, invoking a user-specified function on each of the elements of the input PCollection to produce zero or more output elements, all of which are collected into the output PCollection.

    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.

    DoFns

    The function to use to process each element is specified by a DoFn<InputT, OutputT>, primarily via its processElement method. The DoFn may also override the default implementations of startBundle and finishBundle.

    Conceptually, when a ParDo transform is executed, the elements of the input PCollection are first divided up into some number of "bundles". These are farmed off to distributed worker machines (or run locally, if using the DirectPipelineRunner). For each bundle of input elements processing proceeds as follows:

    1. A fresh instance of the argument DoFn is created on a worker. This may be through deserialization or other means. If the DoFn subclass does not override startBundle or finishBundle then this may be optimized since it cannot observe the start and end of a bundle.
    2. The DoFn's DoFn.startBundle(com.google.cloud.dataflow.sdk.transforms.DoFn<InputT, OutputT>.Context) method is called to initialize it. If this method is not overridden, the call may be optimized away.
    3. The DoFn's DoFn.processElement(com.google.cloud.dataflow.sdk.transforms.DoFn<InputT, OutputT>.ProcessContext) method is called on each of the input elements in the bundle.
    4. The DoFn's DoFn.finishBundle(com.google.cloud.dataflow.sdk.transforms.DoFn<InputT, OutputT>.Context) method is called to complete its work. After DoFn.finishBundle(com.google.cloud.dataflow.sdk.transforms.DoFn<InputT, OutputT>.Context) is called, the framework will never again invoke any of these three processing methods. If this method is not overridden, this call may be optimized away.

    Each of the calls to any of the DoFn's processing methods can produce zero or more output elements. All of the of output elements from all of the DoFn instances are included in the output PCollection.

    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 has the same WindowFn associated with it as the input.

    Naming ParDo transforms

    The name of a transform is used to provide a name for any node in the Pipeline graph resulting from application of the transform. It is best practice to provide a name at the time of application, via PCollection.apply(String, PTransform). Otherwise, a unique name - which may not be stable across pipeline revision - will be generated, based on the transform name.

    If a ParDo is applied exactly once inlined, then it can be given a name via named(java.lang.String). 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>() { ... }));
      

    Side Inputs

    While a ParDo processes elements from 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.withSideInputs(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);
               }
             }}));
      

    Side Outputs

    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 are specified by invoking withOutputTags(com.google.cloud.dataflow.sdk.values.TupleTag<OutputT>, com.google.cloud.dataflow.sdk.values.TupleTagList). Unconsumed side outputs do not necessarily need to be explicitly 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);
      

    Properties May Be Specified In Any Order

    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>).

    Output Coders

    By default, the Coder<OutputT> for the elements of the main output PCollection<OutputT> is inferred from the concrete type of the DoFn<InputT, OutputT>.

    By default, the Coder<SideOutputT> for the elements of a side output PCollection<SideOutputT> is inferred from the concrete type of the corresponding TupleTag<SideOutputT>. 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.

    Serializability of DoFns

    A 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 bundles 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 bundle 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 bundle.

    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:

    • Define the DoFn as a named, static class.
    • Define the DoFn as an anonymous inner class inside of a static method.

    Both of these approaches ensure that there is no implicit enclosing 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 bundle:

    • Define instance variable state (including implicit instance variables holding final variables captured by an anonymous inner class), initialized by the 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 bundle. This method is good for state known when the original DoFn is created in the main program, if it's not overly large.
    • Compute the state as a singleton 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.
    • Initialize the state in each 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.

    No Global Shared State

    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.

    Fault Tolerance

    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 bundle. This means that a DoFn instance might process a bundle partially, then crash for some reason, then be rerun (often on a different worker machine) on that same bundle and on the same elements as before. Sometimes two or more DoFn instances will be running on the same bundle 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, such as 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.

    Optimization

    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.

    See Also:
    the web documentation for ParDo


Was this page helpful? Let us know how we did:

Send feedback about...

Cloud Dataflow
Need help? Visit our support page.