InputT
- type of input valuesAccumT
- type of mutable accumulator valuesOutputT
- type of output valuespublic abstract static class Combine.CombineFn<InputT,AccumT,OutputT>
extends java.lang.Object
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:
InputT
values are partitioned into one or more
batches.
createAccumulator()
operation is
invoked to create a fresh mutable accumulator value of type
AccumT
, initialized to represent the combination of zero
values.
InputT
value in a batch, the
addInput(AccumT, InputT)
operation is invoked to add the value to that
batch's accumulator AccumT
value. The accumulator may just
record the new value (e.g., if AccumT == List<InputT>
, or may do
work to represent the combination more compactly.
mergeAccumulators(java.lang.Iterable<AccumT>)
operation is invoked to
combine a collection of accumulator AccumT
values into a
single combined output accumulator AccumT
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.
extractOutput(AccumT)
operation is invoked on the final
accumulator AccumT
value to get the output OutputT
value.
For example:
public class AverageFn extends CombineFn<Integer, AverageFn.Accum, Double> {
public static class Accum {
int sum = 0;
int count = 0;
}
public Accum createAccumulator() {
return new Accum();
}
public Accum addInput(Accum accum, Integer input) {
accum.sum += input;
accum.count++;
return accum;
}
public Accum mergeAccumulators(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.
Constructor and Description |
---|
CombineFn() |
Modifier and Type | Method and Description |
---|---|
abstract AccumT |
addInput(AccumT accumulator,
InputT input)
Adds the given input value to the given accumulator, returning the
new accumulator value.
|
OutputT |
apply(java.lang.Iterable<? extends InputT> inputs)
Applies this
CombineFn to a collection of input values
to produce a combined output value. |
AccumT |
compact(AccumT accumulator)
Returns an accumulator that represents the same logical value as the
input accumulator, but may have a more compact representation.
|
abstract AccumT |
createAccumulator()
Returns a new, mutable accumulator value, representing the accumulation of zero input values.
|
OutputT |
defaultValue()
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.
|
java.lang.reflect.TypeVariable<?> |
getAccumTVariable()
Returns the
TypeVariable of AccumT . |
Coder<AccumT> |
getAccumulatorCoder(CoderRegistry registry,
Coder<InputT> inputCoder)
Returns the
Coder to use for accumulator AccumT
values, or null if it is not able to be inferred. |
Coder<OutputT> |
getDefaultOutputCoder(CoderRegistry registry,
Coder<InputT> inputCoder)
Returns the
Coder to use by default for output
OutputT values, or null if it is not able to be inferred. |
java.lang.String |
getIncompatibleGlobalWindowErrorMessage()
Returns the error message for not supported default values in Combine.globally().
|
java.lang.reflect.TypeVariable<?> |
getInputTVariable()
Returns the
TypeVariable of InputT . |
java.lang.reflect.TypeVariable<?> |
getOutputTVariable()
Returns the
TypeVariable of OutputT . |
TypeDescriptor<OutputT> |
getOutputType()
Returns a
TypeDescriptor capturing what is known statically
about the output type of this CombineFn instance's
most-derived class. |
abstract AccumT |
mergeAccumulators(java.lang.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.
|
public abstract AccumT createAccumulator()
public abstract AccumT addInput(AccumT accumulator, InputT input)
For efficiency, the input accumulator may be modified and returned.
public abstract AccumT mergeAccumulators(java.lang.Iterable<AccumT> accumulators)
May modify any of the argument accumulators. May return a fresh accumulator, or may return one of the (modified) argument accumulators.
public abstract OutputT extractOutput(AccumT accumulator)
public AccumT compact(AccumT accumulator)
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.
public OutputT apply(java.lang.Iterable<? extends InputT> inputs)
CombineFn
to a collection of input values
to produce a combined output value.
Useful when using a CombineFn
separately from a
Combine
transform. Does not invoke the
mergeAccumulators(java.lang.Iterable<AccumT>)
operation.
public OutputT defaultValue()
By default returns the extract output of an empty accumulator.
public TypeDescriptor<OutputT> getOutputType()
TypeDescriptor
capturing what is known statically
about the output type of this CombineFn
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.
public Coder<AccumT> getAccumulatorCoder(CoderRegistry registry, Coder<InputT> inputCoder) throws CannotProvideCoderException
CombineFnBase.GlobalCombineFn
Coder
to use for accumulator AccumT
values, or null if it is not able to be inferred.
By default, uses the knowledge of the Coder
being used
for InputT
values and the enclosing Pipeline
's
CoderRegistry
to try to infer the Coder for AccumT
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.
getAccumulatorCoder
in interface CombineFnBase.GlobalCombineFn<InputT,AccumT,OutputT>
CannotProvideCoderException
public Coder<OutputT> getDefaultOutputCoder(CoderRegistry registry, Coder<InputT> inputCoder) throws CannotProvideCoderException
CombineFnBase.GlobalCombineFn
Coder
to use by default for output
OutputT
values, or null if it is not able to be inferred.
By default, uses the knowledge of the Coder
being
used for input InputT
values and the enclosing
Pipeline
's CoderRegistry
to try to infer the
Coder for OutputT
values.
getDefaultOutputCoder
in interface CombineFnBase.GlobalCombineFn<InputT,AccumT,OutputT>
CannotProvideCoderException
public java.lang.String getIncompatibleGlobalWindowErrorMessage()
CombineFnBase.GlobalCombineFn
getIncompatibleGlobalWindowErrorMessage
in interface CombineFnBase.GlobalCombineFn<InputT,AccumT,OutputT>
public java.lang.reflect.TypeVariable<?> getInputTVariable()
TypeVariable
of InputT
.public java.lang.reflect.TypeVariable<?> getAccumTVariable()
TypeVariable
of AccumT
.public java.lang.reflect.TypeVariable<?> getOutputTVariable()
TypeVariable
of OutputT
.public void populateDisplayData(DisplayData.Builder builder)
populateDisplayData(DisplayData.Builder)
is invoked by Pipeline runners to collect
display data via DisplayData.from(HasDisplayData)
. Implementations may call
super.populateDisplayData(builder)
in order to register display data in the current
namespace, but should otherwise use subcomponent.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.
populateDisplayData
in interface HasDisplayData
builder
- The builder to populate with display data.HasDisplayData