Class Combine.CombineFn<InputT extends @Nullable Object,AccumT extends @Nullable Object,OutputT extends @Nullable Object>
- Type Parameters:
InputT
- type of input valuesAccumT
- type of mutable accumulator valuesOutputT
- type of output values
- All Implemented Interfaces:
Serializable
,CombineFnBase.GlobalCombineFn<InputT,
,AccumT, OutputT> HasDisplayData
- Direct Known Subclasses:
ApproximateDistinct.ApproximateDistinctFn
,ApproximateUnique.ApproximateUniqueCombineFn
,ArrayAgg.ArrayAggArray
,BeamBuiltinAggregations.BitXOr
,BeamBuiltinAnalyticFunctions.PositionAwareCombineFn
,Combine.AccumulatingCombineFn
,Combine.BinaryCombineDoubleFn
,Combine.BinaryCombineFn
,Combine.BinaryCombineIntegerFn
,Combine.BinaryCombineLongFn
,Combine.IterableCombineFn
,CombineFns.ComposedCombineFn
,CountIf.CountIfFn
,CovarianceFn
,DefaultSequenceCombiner
,FlinkStreamingAggregationsTranslators.ConcatenateAsIterable
,LazyAggregateCombineFn
,Sample.FixedSizedSampleFn
,SketchFrequencies.CountMinSketchFn
,SnowflakeIO.Concatenate
,StringAgg.StringAggByte
,StringAgg.StringAggString
,TDigestQuantiles.TDigestQuantilesFn
,TypedCombineFnDelegate
,VarianceFn
- Enclosing class:
Combine
CombineFn<InputT, AccumT, OutputT>
specifies how to combine a collection of input
values of type InputT
into a single output value of type OutputT
. It does this
via one or more intermediate mutable accumulator values of type AccumT
.
The overall process to combine a collection of input InputT
values into a single
output OutputT
value is as follows:
- The input
InputT
values are partitioned into one or more batches. - For each batch, the
createAccumulator()
operation is invoked to create a fresh mutable accumulator value of typeAccumT
, initialized to represent the combination of zero values. - For each input
InputT
value in a batch, theaddInput(AccumT, InputT)
operation is invoked to add the value to that batch's accumulatorAccumT
value. The accumulator may just record the new value (e.g., ifAccumT == List<InputT>
, or may do work to represent the combination more compactly. - The
mergeAccumulators(java.lang.Iterable<AccumT>)
operation is invoked to combine a collection of accumulatorAccumT
values into a single combined output accumulatorAccumT
value, once the merging accumulators have had all all the input values in their batches added to them. This operation is invoked repeatedly, until there is only one accumulator value left. - The
extractOutput(AccumT)
operation is invoked on the final accumulatorAccumT
value to get the outputOutputT
value.
For example:
public class AverageFn extendsCombineFn<Integer, AverageFn.Accum, Double> {
public static class Accum implements Serializable {
int sum = 0;
int count = 0;
@Override
public boolean equals(@Nullable Object other) {
if (other == null) return false;
if (other == this) return true;
if (!(other instanceof Accum))return false;
Accum o = (Accum)other;
if (this.sum != o.sum || this.count != o.count) {
return false;
} else {
return true;
}
}
}
public Accum createAccumulator() {
return new Accum();
}
public Accum addInput(Accum accum, Integer input) {
accum.sum += input;
accum.count++;
return accum;
}
public AccummergeAccumulators(Iterable<Accum> accums) {
Accum merged = createAccumulator();
for (Accum accum : accums) {
merged.sum += accum.sum;
merged.count += accum.count;
}
return merged;
}
public Double extractOutput(Accum accum) {
return ((double) accum.sum) / accum.count;
}
}
PCollection<Integer> pc = ...;
PCollection<Double> average = pc.apply(Combine.globally(new AverageFn()));
Combining functions used by Combine.Globally
, Combine.PerKey
, Combine.GroupedValues
, and PTransforms
derived from them should be associative
and commutative. Associativity is required because input values are first broken up into
subgroups before being combined, and their intermediate results further combined, in an
arbitrary tree structure. Commutativity is required because any order of the input values is
ignored when breaking up input values into groups.
Note on Data Encoding
Some form of data encoding is required when using custom types in a CombineFn which do not
have well-known coders. The sample code above uses a custom Accumulator which gets coder by
implementing Serializable
. By doing this, we are relying on the generic CoderProvider
, which is able to provide a coder for any Serializable
if applicable. In cases where Serializable
is not
efficient, or inapplicable, in general there are two alternatives for encoding:
- Default
CoderRegistry
. For example, implement a coder class explicitly and use the@DefaultCoder
tag. See theCoderRegistry
for the numerous ways in which to bind a type to a coder. - CombineFn specific way. While extending CombineFn, overwrite both
CombineFnBase.GlobalCombineFn.getAccumulatorCoder(org.apache.beam.sdk.coders.CoderRegistry, org.apache.beam.sdk.coders.Coder<InputT>)
andCombineFnBase.GlobalCombineFn.getDefaultOutputCoder(org.apache.beam.sdk.coders.CoderRegistry, org.apache.beam.sdk.coders.Coder<InputT>)
.
- See Also:
-
Constructor Summary
Constructors -
Method Summary
Modifier and TypeMethodDescriptionabstract AccumT
Adds the given input value to the given accumulator, returning the new accumulator value.Applies thisCombineFn
to a collection of input values to produce a combined output value.Returns an accumulator that represents the same logical value as the input accumulator, but may have a more compact representation.abstract AccumT
Returns a new, mutable accumulator value, representing the accumulation of zero input values.Returns the default value when there are no values added to the accumulator.abstract OutputT
extractOutput
(AccumT accumulator) Returns the output value that is the result of combining all the input values represented by the given accumulator.TypeVariable
<?> Returns theTypeVariable
ofAccumT
.getAccumulatorCoder
(CoderRegistry registry, Coder<InputT> inputCoder) Returns theCoder
to use for accumulatorAccumT
values, or null if it is not able to be inferred.getDefaultOutputCoder
(CoderRegistry registry, Coder<InputT> inputCoder) Returns theCoder
to use by default for outputOutputT
values, or null if it is not able to be inferred.Returns the error message for not supported default values in Combine.globally().TypeVariable
<?> Returns theTypeVariable
ofInputT
.Returns aTypeDescriptor
capturing what is known statically about the input type of thisCombineFn
instance's most-derived class.TypeVariable
<?> Returns theTypeVariable
ofOutputT
.Returns aTypeDescriptor
capturing what is known statically about the output type of thisCombineFn
instance's most-derived class.abstract AccumT
mergeAccumulators
(Iterable<AccumT> accumulators) Returns an accumulator representing the accumulation of all the input values accumulated in the merging accumulators.void
populateDisplayData
(DisplayData.Builder builder) Register display data for the given transform or component.
-
Constructor Details
-
CombineFn
public CombineFn()
-
-
Method Details
-
createAccumulator
Returns a new, mutable accumulator value, representing the accumulation of zero input values. -
addInput
Adds the given input value to the given accumulator, returning the new accumulator value.- Parameters:
mutableAccumulator
- may be modified and returned for efficiencyinput
- should not be mutated
-
mergeAccumulators
Returns an accumulator representing the accumulation of all the input values accumulated in the merging accumulators.- Parameters:
accumulators
- only the first accumulator may be modified and returned for efficiency; the other accumulators should not be mutated, because they may be shared with other code and mutating them could lead to incorrect results or data corruption.
-
extractOutput
Returns the output value that is the result of combining all the input values represented by the given accumulator.- Parameters:
accumulator
- can be modified for efficiency
-
compact
Returns an accumulator that represents the same logical value as the input accumulator, but may have a more compact representation.For most CombineFns this would be a no-op, but should be overridden by CombineFns that (for example) buffer up elements and combine them in batches.
For efficiency, the input accumulator may be modified and returned.
By default returns the original accumulator.
-
apply
Applies thisCombineFn
to a collection of input values to produce a combined output value.Useful when using a
CombineFn
separately from aCombine
transform. Does not invoke themergeAccumulators(java.lang.Iterable<AccumT>)
operation. -
defaultValue
Returns the default value when there are no values added to the accumulator.By default returns the extract output of an empty accumulator.
-
getOutputType
Returns aTypeDescriptor
capturing what is known statically about the output type of thisCombineFn
instance's most-derived class.In the normal case of a concrete
CombineFn
subclass with no generic type parameters of its own, this will be a complete non-generic type. -
getInputType
Returns aTypeDescriptor
capturing what is known statically about the input type of thisCombineFn
instance's most-derived class.In the normal case of a concrete
CombineFn
subclass with no generic type parameters of its own, this will be a complete non-generic type. -
getAccumulatorCoder
public Coder<AccumT> getAccumulatorCoder(CoderRegistry registry, Coder<InputT> inputCoder) throws CannotProvideCoderException Description copied from interface:CombineFnBase.GlobalCombineFn
Returns theCoder
to use for accumulatorAccumT
values, or null if it is not able to be inferred.By default, uses the knowledge of the
Coder
being used forInputT
values and the enclosingPipeline
'sCoderRegistry
to try to infer the Coder forAccumT
values.This is the Coder used to send data through a communication-intensive shuffle step, so a compact and efficient representation may have significant performance benefits.
- Specified by:
getAccumulatorCoder
in interfaceCombineFnBase.GlobalCombineFn<InputT,
AccumT, OutputT> - Throws:
CannotProvideCoderException
-
getDefaultOutputCoder
public Coder<OutputT> getDefaultOutputCoder(CoderRegistry registry, Coder<InputT> inputCoder) throws CannotProvideCoderException Description copied from interface:CombineFnBase.GlobalCombineFn
Returns theCoder
to use by default for outputOutputT
values, or null if it is not able to be inferred.By default, uses the knowledge of the
Coder
being used for inputInputT
values and the enclosingPipeline
'sCoderRegistry
to try to infer the Coder forOutputT
values.- Specified by:
getDefaultOutputCoder
in interfaceCombineFnBase.GlobalCombineFn<InputT,
AccumT, OutputT> - Throws:
CannotProvideCoderException
-
getIncompatibleGlobalWindowErrorMessage
Description copied from interface:CombineFnBase.GlobalCombineFn
Returns the error message for not supported default values in Combine.globally().- Specified by:
getIncompatibleGlobalWindowErrorMessage
in interfaceCombineFnBase.GlobalCombineFn<InputT,
AccumT, OutputT>
-
getInputTVariable
Returns theTypeVariable
ofInputT
. -
getAccumTVariable
Returns theTypeVariable
ofAccumT
. -
getOutputTVariable
Returns theTypeVariable
ofOutputT
. -
populateDisplayData
Register display data for the given transform or component.populateDisplayData(DisplayData.Builder)
is invoked by Pipeline runners to collect display data viaDisplayData.from(HasDisplayData)
. Implementations may callsuper.populateDisplayData(builder)
in order to register display data in the current namespace, but should otherwise usesubcomponent.populateDisplayData(builder)
to use the namespace of the subcomponent.By default, does not register any display data. Implementors may override this method to provide their own display data.
- Specified by:
populateDisplayData
in interfaceHasDisplayData
- Parameters:
builder
- The builder to populate with display data.- See Also:
-