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 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 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.
 
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:
 
CoderRegistry. For example, implement a coder
       class explicitly and use the @DefaultCoder tag. See the CoderRegistry for the numerous ways in which to bind a type
       to a coder.
   getAccumulatorCoder(org.apache.beam.sdk.coders.CoderRegistry, org.apache.beam.sdk.coders.Coder<InputT>) and getDefaultOutputCoder(org.apache.beam.sdk.coders.CoderRegistry, org.apache.beam.sdk.coders.Coder<InputT>).
 | Constructor and Description | 
|---|
| CombineFn() | 
| Modifier and Type | Method and Description | 
|---|---|
| abstract AccumT | addInput(AccumT mutableAccumulator,
        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  CombineFnto 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  TypeVariableofAccumT. | 
| Coder<AccumT> | getAccumulatorCoder(CoderRegistry registry,
                   Coder<InputT> inputCoder)Returns the  Coderto use for accumulatorAccumTvalues, or null if it is not
 able to be inferred. | 
| Coder<OutputT> | getDefaultOutputCoder(CoderRegistry registry,
                     Coder<InputT> inputCoder)Returns the  Coderto use by default for outputOutputTvalues, 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  TypeVariableofInputT. | 
| TypeDescriptor<InputT> | getInputType()Returns a  TypeDescriptorcapturing what is known statically about the input type of
 thisCombineFninstance's most-derived class. | 
| java.lang.reflect.TypeVariable<?> | getOutputTVariable()Returns the  TypeVariableofOutputT. | 
| TypeDescriptor<OutputT> | getOutputType()Returns a  TypeDescriptorcapturing what is known statically about the output type of
 thisCombineFninstance'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 mutableAccumulator, InputT input)
mutableAccumulator - may be modified and returned for efficiencyinput - should not be mutatedpublic abstract AccumT mergeAccumulators(java.lang.Iterable<AccumT> accumulators)
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.public abstract OutputT extractOutput(AccumT accumulator)
accumulator - can be modified for efficiencypublic 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 TypeDescriptor<InputT> getInputType()
TypeDescriptor capturing what is known statically about the input 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.GlobalCombineFnCoder 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>CannotProvideCoderExceptionpublic Coder<OutputT> getDefaultOutputCoder(CoderRegistry registry, Coder<InputT> inputCoder) throws CannotProvideCoderException
CombineFnBase.GlobalCombineFnCoder 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>CannotProvideCoderExceptionpublic java.lang.String getIncompatibleGlobalWindowErrorMessage()
CombineFnBase.GlobalCombineFngetIncompatibleGlobalWindowErrorMessage 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 HasDisplayDatabuilder - The builder to populate with display data.HasDisplayData