apache_beam.transforms.ptransform module¶
PTransform and descendants.
A PTransform is an object describing (not executing) a computation. The actual execution semantics for a transform is captured by a runner object. A transform object always belongs to a pipeline object.
A PTransform derived class needs to define the expand() method that describes how one or more PValues are created by the transform.
The module defines a few standard transforms: FlatMap (parallel do), GroupByKey (group by key), etc. Note that the expand() methods for these classes contain code that will add nodes to the processing graph associated with a pipeline.
As support for the FlatMap transform, the module also defines a DoFn class and wrapper class that allows lambda functions to be used as FlatMap processing functions.
-
class
apache_beam.transforms.ptransform.
PTransform
(label=None)[source]¶ Bases:
apache_beam.typehints.decorators.WithTypeHints
,apache_beam.transforms.display.HasDisplayData
A transform object used to modify one or more PCollections.
Subclasses must define an expand() method that will be used when the transform is applied to some arguments. Typical usage pattern will be:
input | CustomTransform(…)The expand() method of the CustomTransform object passed in will be called with input as an argument.
-
side_inputs
= ()¶
-
pipeline
= None¶
-
label
¶
-
with_input_types
(input_type_hint)[source]¶ Annotates the input type of a
PTransform
with a type-hint.Parameters: input_type_hint (type) – An instance of an allowed built-in type, a custom class, or an instance of a TypeConstraint
.Raises: TypeError
– If input_type_hint is not a valid type-hint. Seeapache_beam.typehints.typehints.validate_composite_type_param()
for further details.Returns: A reference to the instance of this particular PTransform
object. This allows chaining type-hinting related methods.Return type: PTransform
-
with_output_types
(type_hint)[source]¶ Annotates the output type of a
PTransform
with a type-hint.Parameters: type_hint (type) – An instance of an allowed built-in type, a custom class, or a TypeConstraint
.Raises: TypeError
– If type_hint is not a valid type-hint. Seevalidate_composite_type_param()
for further details.Returns: A reference to the instance of this particular PTransform
object. This allows chaining type-hinting related methods.Return type: PTransform
-
-
apache_beam.transforms.ptransform.
ptransform_fn
(fn)[source]¶ A decorator for a function-based PTransform.
Experimental; no backwards-compatibility guarantees.
Parameters: fn – A function implementing a custom PTransform. Returns: A CallablePTransform instance wrapping the function-based PTransform. This wrapper provides an alternative, simpler way to define a PTransform. The standard method is to subclass from PTransform and override the expand() method. An equivalent effect can be obtained by defining a function that accepts an input PCollection and additional optional arguments and returns a resulting PCollection. For example:
@ptransform_fn def CustomMapper(pcoll, mapfn): return pcoll | ParDo(mapfn)
The equivalent approach using PTransform subclassing:
class CustomMapper(PTransform): def __init__(self, mapfn): super(CustomMapper, self).__init__() self.mapfn = mapfn def expand(self, pcoll): return pcoll | ParDo(self.mapfn)
With either method the custom PTransform can be used in pipelines as if it were one of the “native” PTransforms:
result_pcoll = input_pcoll | 'Label' >> CustomMapper(somefn)
Note that for both solutions the underlying implementation of the pipe operator (i.e., |) will inject the pcoll argument in its proper place (first argument if no label was specified and second argument otherwise).