Package org.apache.beam.sdk.transforms
Annotation Interface DoFn.ProcessElement
Annotation for the method to use for processing elements. A subclass of
DoFn must have
a method with this annotation.
If any of the arguments is a RestrictionTracker then see the specifications below
about splittable DoFn, otherwise this method must satisfy the following constraints:
- If one of its arguments is tagged with the
DoFn.Elementannotation, then it will be passed the current element being processed. The argument type must match the input type of this DoFn exactly, or both types must have equivalent schemas registered. - If one of its arguments is tagged with the
DoFn.Timestampannotation, then it will be passed the timestamp of the current element being processed; the argument must be of typeInstant. - If one of its arguments is a subtype of
BoundedWindow, then it will be passed the window of the current element. When applied byParDothe subtype ofBoundedWindowmust match the type of windows on the inputPCollection. If the window is not accessed a runner may perform additional optimizations. - If one of its arguments is of type
PaneInfo, then it will be passed information about the current triggering pane. - If one of the parameters is of type
PipelineOptions, then it will be passed the options for the current pipeline. - If one of the parameters is of type
DoFn.OutputReceiver, then it will be passed an output receiver for outputting elements to the default output. - If one of the parameters is of type
DoFn.MultiOutputReceiver, then it will be passed an output receiver for outputting to multiple tagged outputs. - If one of the parameters is of type
DoFn.BundleFinalizer, then it will be passed a mechanism to register a callback that will be invoked after the runner successfully commits the output of this bundle. See Apache Beam Portability API: How to Finalize Bundles for further details. - It must return
void.
Splittable DoFn's
A DoFn is splittable if its DoFn.ProcessElement method has a parameter
whose type is of RestrictionTracker. This is an advanced feature and an overwhelming
majority of users will never need to write a splittable DoFn.
Not all runners support Splittable DoFn. See the capability matrix.
See the proposal for an overview of the involved concepts (splittable DoFn, restriction, restriction tracker).
A splittable DoFn must obey the following constraints:
- The type of restrictions used by all of these methods must be the same.
- It must define a
DoFn.GetInitialRestrictionmethod. - It may define a
DoFn.GetSizemethod or ensure that theRestrictionTrackerimplementsRestrictionTracker.HasProgress. Poor auto-scaling of workers and/or splitting may result if size or progress is an inaccurate representation of work. SeeDoFn.GetSizeandRestrictionTracker.HasProgressfor further details. - It should define a
DoFn.SplitRestrictionmethod. This method enables runners to perform bulk splitting initially allowing for a rapid increase in parallelism. If it is not defined, there is no initial split happening by default. Note that initial split is a different concept from the split during element processing time. SeeRestrictionTracker.trySplit(double)for details about splitting when the current element and restriction are actively being processed. - It may define a
DoFn.TruncateRestrictionmethod to choose how to truncate a restriction such that it represents a finite amount of work when the pipeline is draining. SeeDoFn.TruncateRestrictionandRestrictionTracker.isBounded()for additional details. - It may define a
DoFn.NewTrackermethod returning a subtype ofRestrictionTracker<R>whereRis the restriction type returned byDoFn.GetInitialRestriction. This method is optional only if the restriction type returned byDoFn.GetInitialRestrictionimplementsHasDefaultTracker. - It may define a
DoFn.GetRestrictionCodermethod. - It may define a
DoFn.GetInitialWatermarkEstimatorStatemethod. If none is defined then the watermark estimator state is of typeVoid. - It may define a
DoFn.GetWatermarkEstimatorStateCodermethod. - It may define a
DoFn.NewWatermarkEstimatormethod returning a subtype ofWatermarkEstimator<W>whereWis the watermark estimator state type returned byDoFn.GetInitialWatermarkEstimatorState. This method is optional only ifDoFn.GetInitialWatermarkEstimatorStatehas not been defined orWimplementsHasDefaultWatermarkEstimator. - The
DoFnitself may be annotated withDoFn.BoundedPerElementorDoFn.UnboundedPerElement, but not both at the same time. If it's not annotated with either of these, it's assumed to beDoFn.BoundedPerElementif itsDoFn.ProcessElementmethod returnsvoidandDoFn.UnboundedPerElementif it returns aDoFn.ProcessContinuation. - Timers and state must not be used.
If this DoFn is splittable, this method must satisfy the following constraints:
- One of its arguments must be a
RestrictionTracker. The argument must be of the exact typeRestrictionTracker<RestrictionT, PositionT>. - If one of its arguments is tagged with the
DoFn.Elementannotation, then it will be passed the current element being processed. The argument type must match the input type of this DoFn exactly, or both types must have equivalent schemas registered. - If one of its arguments is tagged with the
DoFn.Restrictionannotation, then it will be passed the current restriction being processed; the argument must be of typeRestrictionT. - If one of its arguments is tagged with the
DoFn.Timestampannotation, then it will be passed the timestamp of the current element being processed; the argument must be of typeInstant. - If one of its arguments is of the type
WatermarkEstimator, then it will be passed the watermark estimator. - If one of its arguments is of the type
ManualWatermarkEstimator, then it will be passed a watermark estimator that can be updated manually. This parameter can only be supplied if the method annotated withDoFn.GetInitialWatermarkEstimatorStatereturns a sub-type ofManualWatermarkEstimator. - If one of its arguments is a subtype of
BoundedWindow, then it will be passed the window of the current element. When applied byParDothe subtype ofBoundedWindowmust match the type of windows on the inputPCollection. If the window is not accessed a runner may perform additional optimizations. - If one of its arguments is of type
PaneInfo, then it will be passed information about the current triggering pane. - If one of the parameters is of type
PipelineOptions, then it will be passed the options for the current pipeline. - If one of the parameters is of type
DoFn.OutputReceiver, then it will be passed an output receiver for outputting elements to the default output. - If one of the parameters is of type
DoFn.MultiOutputReceiver, then it will be passed an output receiver for outputting to multiple tagged outputs. - If one of the parameters is of type
DoFn.BundleFinalizer, then it will be passed a mechanism to register a callback that will be invoked after the runner successfully commits the output of this bundle. See Apache Beam Portability API: How to Finalize Bundles for further details. - May return a
DoFn.ProcessContinuationto indicate whether there is more work to be done for the current element, otherwise must returnvoid.