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

  • Type Parameters:
    InputT - type of input values
    AccumT - type of mutable accumulator values
    OutputT - type of output values
    All Implemented Interfaces:
    java.io.Serializable, CombineFnBase.GlobalCombineFn<InputT,​AccumT,​OutputT>, HasDisplayData
    Direct Known Subclasses:
    ApproximateUnique.ApproximateUniqueCombineFn, Combine.AccumulatingCombineFn, Combine.BinaryCombineDoubleFn, Combine.BinaryCombineFn, Combine.BinaryCombineIntegerFn, Combine.BinaryCombineLongFn, Combine.IterableCombineFn, CombineFns.ComposedCombineFn, Sample.FixedSizedSampleFn
    Enclosing class:
    Combine

    public abstract static class Combine.CombineFn<InputT extends @Nullable java.lang.Object,​AccumT extends @Nullable java.lang.Object,​OutputT extends @Nullable java.lang.Object>
    extends java.lang.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:
    Serialized Form
    • Constructor Summary

      Constructors 
      Constructor Description
      CombineFn()  
    • Method Summary

      All Methods Instance Methods Abstract Methods Concrete Methods 
      Modifier and Type Method 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 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.
      TypeDescriptor<InputT> getInputType()
      Returns a TypeDescriptor capturing what is known statically about the input type of this CombineFn instance's most-derived class.
      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.
      • Methods inherited from class java.lang.Object

        clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
    • Constructor Detail

      • CombineFn

        public CombineFn()
    • Method Detail

      • 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​(java.lang.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​(java.lang.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
      • getInputTVariable

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

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

        public java.lang.reflect.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:
        HasDisplayData