Class Combine.AccumulatingCombineFn<InputT,AccumT extends Combine.AccumulatingCombineFn.Accumulator<InputT,AccumT,OutputT>,OutputT>
- 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:
ApproximateQuantiles.ApproximateQuantilesCombineFn,Top.TopCombineFn
- Enclosing class:
Combine
CombineFn that uses a subclass of Combine.AccumulatingCombineFn.Accumulator as its
accumulator type. By defining the operations of the Accumulator helper class, the
operations of the enclosing CombineFn are automatically provided. This can reduce the
code required to implement a CombineFn.
For example, the example from Combine.CombineFn above can be expressed using
AccumulatingCombineFn more concisely as follows:
public class AverageFn
extends AccumulatingCombineFn<Integer, AverageFn.Accum, Double> {
public Accum createAccumulator() {
return new Accum();
}
public class Accum
extends AccumulatingCombineFn<Integer, AverageFn.Accum, Double>
.Accumulator {
private int sum = 0;
private int count = 0;
public void addInput(Integer input) {
sum += input;
count++;
}
public void mergeAccumulator(Accum other) {
sum += other.sum;
count += other.count;
}
public Double extractOutput() {
return ((double) sum) / count;
}
}
}
PCollection<Integer> pc = ...;
PCollection<Double> average = pc.apply(Combine.globally(new AverageFn()));
- See Also:
-
Nested Class Summary
Nested ClassesModifier and TypeClassDescriptionstatic interfaceThe type of mutable accumulator values used by thisAccumulatingCombineFn. -
Constructor Summary
Constructors -
Method Summary
Modifier and TypeMethodDescriptionfinal AccumTAdds the given input value to the given accumulator, returning the new accumulator value.final OutputTextractOutput(AccumT accumulator) Returns the output value that is the result of combining all the input values represented by the given accumulator.TypeVariable<?> Returns theTypeVariableofAccumT.getAccumulatorCoder(CoderRegistry registry, Coder<InputT> inputCoder) Returns theCoderto use for accumulatorAccumTvalues, or null if it is not able to be inferred.getDefaultOutputCoder(CoderRegistry registry, Coder<InputT> inputCoder) Returns theCoderto use by default for outputOutputTvalues, or null if it is not able to be inferred.Returns the error message for not supported default values in Combine.globally().TypeVariable<?> Returns theTypeVariableofInputT.TypeVariable<?> Returns theTypeVariableofOutputT.final AccumTmergeAccumulators(Iterable<AccumT> accumulators) Returns an accumulator representing the accumulation of all the input values accumulated in the merging accumulators.voidpopulateDisplayData(DisplayData.Builder builder) Register display data for the given transform or component.Methods inherited from class org.apache.beam.sdk.transforms.Combine.CombineFn
apply, compact, createAccumulator, defaultValue, getInputType, getOutputType
-
Constructor Details
-
AccumulatingCombineFn
public AccumulatingCombineFn()
-
-
Method Details
-
addInput
Description copied from class:Combine.CombineFnAdds the given input value to the given accumulator, returning the new accumulator value.- Specified by:
addInputin classCombine.CombineFn<InputT,AccumT extends Combine.AccumulatingCombineFn.Accumulator<InputT, AccumT, OutputT>, OutputT> - Parameters:
accumulator- may be modified and returned for efficiencyinput- should not be mutated
-
mergeAccumulators
Description copied from class:Combine.CombineFnReturns an accumulator representing the accumulation of all the input values accumulated in the merging accumulators.- Specified by:
mergeAccumulatorsin classCombine.CombineFn<InputT,AccumT extends Combine.AccumulatingCombineFn.Accumulator<InputT, AccumT, OutputT>, OutputT> - 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
Description copied from class:Combine.CombineFnReturns the output value that is the result of combining all the input values represented by the given accumulator.- Specified by:
extractOutputin classCombine.CombineFn<InputT,AccumT extends Combine.AccumulatingCombineFn.Accumulator<InputT, AccumT, OutputT>, OutputT> - Parameters:
accumulator- can be modified for efficiency
-
getAccumulatorCoder
public Coder<AccumT> getAccumulatorCoder(CoderRegistry registry, Coder<InputT> inputCoder) throws CannotProvideCoderException Description copied from interface:CombineFnBase.GlobalCombineFnReturns theCoderto use for accumulatorAccumTvalues, or null if it is not able to be inferred.By default, uses the knowledge of the
Coderbeing used forInputTvalues and the enclosingPipeline'sCoderRegistryto try to infer the Coder forAccumTvalues.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:
getAccumulatorCoderin interfaceCombineFnBase.GlobalCombineFn<InputT,AccumT, OutputT> - Throws:
CannotProvideCoderException
-
getDefaultOutputCoder
public Coder<OutputT> getDefaultOutputCoder(CoderRegistry registry, Coder<InputT> inputCoder) throws CannotProvideCoderException Description copied from interface:CombineFnBase.GlobalCombineFnReturns theCoderto use by default for outputOutputTvalues, or null if it is not able to be inferred.By default, uses the knowledge of the
Coderbeing used for inputInputTvalues and the enclosingPipeline'sCoderRegistryto try to infer the Coder forOutputTvalues.- Specified by:
getDefaultOutputCoderin interfaceCombineFnBase.GlobalCombineFn<InputT,AccumT, OutputT> - Throws:
CannotProvideCoderException
-
getIncompatibleGlobalWindowErrorMessage
Description copied from interface:CombineFnBase.GlobalCombineFnReturns the error message for not supported default values in Combine.globally().- Specified by:
getIncompatibleGlobalWindowErrorMessagein interfaceCombineFnBase.GlobalCombineFn<InputT,AccumT, OutputT>
-
getInputTVariable
Returns theTypeVariableofInputT. -
getAccumTVariable
Returns theTypeVariableofAccumT. -
getOutputTVariable
Returns theTypeVariableofOutputT. -
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:
populateDisplayDatain interfaceHasDisplayData- Parameters:
builder- The builder to populate with display data.- See Also:
-