Class Combine.CombineFn<InputT extends @Nullable Object,AccumT extends @Nullable Object,OutputT extends @Nullable Object>

java.lang.Object
org.apache.beam.sdk.transforms.Combine.CombineFn<InputT,AccumT,OutputT>
Type Parameters:
InputT - type of input values
AccumT - type of mutable accumulator values
OutputT - 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

public abstract static class Combine.CombineFn<InputT extends @Nullable Object,AccumT extends @Nullable Object,OutputT extends @Nullable Object> extends Object
A 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:

  1. The input InputT values are partitioned into one or more batches.
  2. For each batch, the createAccumulator() operation is invoked to create a fresh mutable accumulator value of type AccumT, initialized to represent the combination of zero values.
  3. For each input 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.
  4. The 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.
  5. The extractOutput(AccumT) operation is invoked on the final accumulator AccumT value to get the output OutputT 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:

See Also:
  • Constructor Details

    • CombineFn

      public CombineFn()
  • Method Details

    • createAccumulator

      public abstract AccumT createAccumulator()
      Returns a new, mutable accumulator value, representing the accumulation of zero input values.
    • addInput

      public abstract AccumT addInput(AccumT mutableAccumulator, InputT input)
      Adds the given input value to the given accumulator, returning the new accumulator value.
      Parameters:
      mutableAccumulator - may be modified and returned for efficiency
      input - should not be mutated
    • mergeAccumulators

      public abstract AccumT mergeAccumulators(Iterable<AccumT> accumulators)
      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

      public abstract OutputT extractOutput(AccumT accumulator)
      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

      public AccumT compact(AccumT accumulator)
      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

      public OutputT apply(Iterable<? extends InputT> inputs)
      Applies this 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.

    • defaultValue

      public OutputT 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

      public TypeDescriptor<OutputT> getOutputType()
      Returns a 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.

    • getInputType

      public TypeDescriptor<InputT> getInputType()
      Returns a 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.

    • getAccumulatorCoder

      public Coder<AccumT> getAccumulatorCoder(CoderRegistry registry, Coder<InputT> inputCoder) throws CannotProvideCoderException
      Description copied from interface: CombineFnBase.GlobalCombineFn
      Returns the 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.

      Specified by:
      getAccumulatorCoder in interface CombineFnBase.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 the 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.

      Specified by:
      getDefaultOutputCoder in interface CombineFnBase.GlobalCombineFn<InputT,AccumT,OutputT>
      Throws:
      CannotProvideCoderException
    • getIncompatibleGlobalWindowErrorMessage

      public String getIncompatibleGlobalWindowErrorMessage()
      Description copied from interface: CombineFnBase.GlobalCombineFn
      Returns the error message for not supported default values in Combine.globally().
      Specified by:
      getIncompatibleGlobalWindowErrorMessage in interface CombineFnBase.GlobalCombineFn<InputT,AccumT,OutputT>
    • getInputTVariable

      public TypeVariable<?> getInputTVariable()
      Returns the TypeVariable of InputT.
    • getAccumTVariable

      public TypeVariable<?> getAccumTVariable()
      Returns the TypeVariable of AccumT.
    • getOutputTVariable

      public TypeVariable<?> getOutputTVariable()
      Returns the TypeVariable of OutputT.
    • populateDisplayData

      public void populateDisplayData(DisplayData.Builder builder)
      Register display data for the given transform or component.

      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.

      Specified by:
      populateDisplayData in interface HasDisplayData
      Parameters:
      builder - The builder to populate with display data.
      See Also: