Class ParDo


  • public class ParDo
    extends java.lang.Object
    ParDo is the core element-wise transform in Apache Beam, 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 provide a StartBundle and finishBundle method.

    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 DirectRunner). For each bundle of input elements processing proceeds as follows:

    1. If required, a fresh instance of the argument DoFn is created on a worker, and the DoFn.Setup method is called on this instance. This may be through deserialization or other means. A PipelineRunner may reuse DoFn instances for multiple bundles. A DoFn that has terminated abnormally (by throwing an Exception) will never be reused.
    2. The DoFn's DoFn.StartBundle method, if provided, is called to initialize it.
    3. The DoFn's DoFn.ProcessElement method is called on each of the input elements in the bundle.
    4. The DoFn's DoFn.FinishBundle method, if provided, is called to complete its work. After DoFn.FinishBundle is called, the framework will not again invoke DoFn.ProcessElement or DoFn.FinishBundle until a new call to DoFn.StartBundle has occurred.
    5. If any of DoFn.Setup, DoFn.StartBundle, DoFn.ProcessElement or DoFn.FinishBundle methods throw an exception, the DoFn.Teardown method, if provided, will be called on the DoFn instance.
    6. If a runner will no longer use a DoFn, the DoFn.Teardown method, if provided, will be called on the discarded instance.
    7. If a bundle requested bundle finalization by registering a bundle finalization callback, the callback will be invoked after the runner has successfully committed the output of a successful bundle.

    Note also that calls to DoFn.Teardown are best effort, and may not be called before a DoFn is discarded in the general case. As a result, use of the DoFn.Teardown method to perform side effects is not appropriate, because the elements that produced the side effect will not be replayed in case of failure, and those side effects are permanently lost.

    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 an output PCollection.

    For example:

    PCollection<String> lines = ...;
     PCollection<String> words =
         lines.apply(ParDo.of(new DoFn<String, String>() {
            @ProcessElement
             public void processElement(@Element String line,
               OutputReceiver<String> r) {
               for (String word : line.split("[^a-zA-Z']+")) {
                 r.output(word);
               }
             }}));
     PCollection<Integer> wordLengths =
         words.apply(ParDo.of(new DoFn<String, Integer>() {
            @ProcessElement
             public void processElement(@Element String word,
               OutputReceiver<Integer> r) {
               Integer length = word.length();
               r.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.

    For example:

    PCollection<String> words =
         lines.apply("ExtractWords", ParDo.of(new DoFn<String, String>() { ... }));
     PCollection<Integer> wordLengths =
         words.apply("ComputeWordLengths", ParDo.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 ParDo.SingleOutput.withSideInputs(org.apache.beam.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.of(new DoFn<String, String>() {
            @ProcessElement
             public void processElement(ProcessContext c) {
                 String word = c.element();
                 int lengthCutOff = c.sideInput(maxWordLengthCutOffView);
                 if (word.length() <= lengthCutOff) {
                     c.output(word);
                 }
             }}).withSideInputs(maxWordLengthCutOffView));
     

    Additional Outputs

    Optionally, a ParDo transform can produce multiple output PCollections, both a "main output" PCollection<OutputT> plus any number of additional 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 ParDo.SingleOutput.withOutputTags(org.apache.beam.sdk.values.TupleTag<OutputT>, org.apache.beam.sdk.values.TupleTagList). Unconsumed 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.WindowedContext.output(Object), while an element is added to any additional output PCollection using DoFn.WindowedContext.output(TupleTag, Object). 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 additional 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
             .of(new DoFn<String, String>() {
                 // Create a tag for the unconsumed output.
                 final TupleTag<String> specialWordsTag =
                     new TupleTag<String>(){};}
                @ProcessElement
                 public void processElement(@Element String word, MultiOutputReceiver r) {
                   if (word.length() <= wordLengthCutOff) {
                     // Emit this short word to the main output.
                     r.get(wordsBelowCutOffTag).output(word);
                   } else {
                     // Emit this long word's length to a specified output.
                     r.get(wordLengthsAboveCutOffTag).output(word.length());
                   }
                   if (word.startsWith("MARKER")) {
                     // Emit this word to a different specified output.
                     r.get(markedWordsTag).output(word);
                   }
                   if (word.startsWith("SPECIAL")) {
                     // Emit this word to the unconsumed output.
                     r.get(specialWordsTag).output(word);
                   }
                 }})
                 // Specify the main and consumed output tags of the
                 // PCollectionTuple result:
             .withOutputTags(wordsBelowCutOffTag,
                 TupleTagList.of(wordLengthsAboveCutOffTag)
                             .and(markedWordsTag)));
     // Extract the PCollection results, by tag.
     PCollection<String> wordsBelowCutOff =
         results.get(wordsBelowCutOffTag);
     PCollection<Integer> wordLengthsAboveCutOff =
         results.get(wordLengthsAboveCutOffTag);
     PCollection<String> markedWords =
         results.get(markedWordsTag);
     

    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<AdditionalOutputT> for the elements of an output PCollection<AdditionalOutputT> is inferred from the concrete type of the corresponding TupleTag<AdditionalOutputT>. 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 an output should be written with "{}",
     // and explicit generic parameter type:
     final TupleTag<String> additionalOutputTag = new TupleTag<String>(){};
     
    This style of TupleTag instantiation is used in the example of ParDos that produce multiple 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 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 some number of bundles of elements to process.

    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 instances created for bundles. This method is good for state known when the original DoFn is created in the main program, if it's not overly large. This is not suitable for any state which must only be used for a single bundle, as DoFn's may be used to process multiple bundles.
    • 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 a DoFn.Setup method. 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 Beam model 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 output PCollections, 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. Beam runners may strive to mask such failures 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 in a new JVM) 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

    Beam runners may choose to apply optimizations to 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.

    When Beam runners apply fusion optimization, 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 Beam makes heavy use of this modular, composable style, trusting to the runner to "flatten out" all the compositions into highly optimized stages.

    See Also:
    the web documentation for ParDo
    • Nested Class Summary

      Nested Classes 
      Modifier and Type Class Description
      static class  ParDo.MultiOutput<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 output PCollections, which are bundled into a result PCollectionTuple.
      static class  ParDo.SingleOutput<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>.
    • Constructor Summary

      Constructors 
      Constructor Description
      ParDo()  
    • Constructor Detail

      • ParDo

        public ParDo()
    • Method Detail

      • of

        public static <InputT,​OutputT> ParDo.SingleOutput<InputT,​OutputT> of​(DoFn<InputT,​OutputT> fn)
        Creates a ParDo PTransform that will invoke the given DoFn function.

        The resulting PTransform is ready to be applied, or further properties can be set on it first.

      • getDoFnSchemaInformation

        @Internal
        public static DoFnSchemaInformation getDoFnSchemaInformation​(DoFn<?,​?> fn,
                                                                     PCollection<?> input)
        Extract information on how the DoFn uses schemas. In particular, if the schema of an element parameter does not match the input PCollection's schema, convert.