Class ApproximateDistinct
PTransforms for computing the approximate number of distinct elements in a stream.
This class relies on the HyperLogLog algorithm, and more precisely HyperLogLog+, the improved version of Google.
References
The implementation comes from Addthis'
Stream-lib library.
The original paper of the HyperLogLog is available here.
A paper from the same authors to have a clearer view of the algorithm is available here.
Google's HyperLogLog+ version is detailed in this paper.
Parameters
Two parameters can be tuned in order to control the computation's accuracy:
- Precision:
p
Controls the accuracy of the estimation. The precision value will have an impact on the number of buckets used to store information about the distinct elements.
In general one can expect a relative error of about1.1 / sqrt(2^p). The value should be of at least 4 to guarantee a minimal accuracy.
By default, the precision is set to12for a relative error of around2%. - Sparse Precision:
sp
Used to create a sparse representation in order to optimize memory and improve accuracy at small cardinalities.
The value ofspshould be greater thanp, but lower than 32.
By default, the sparse representation is not used (sp = 0). One should use it if the cardinality may be less than12000.
Examples
There are 2 ways of using this class:
- Use the
PTransforms that returnPCollection<Long>corresponding to the estimate number of distinct elements in the inputPCollectionof objects or for each key in aPCollectionofKVs. - Use the
ApproximateDistinct.ApproximateDistinctFnCombineFnthat is exposed in order to make advanced processing involving theHyperLogLogPlusstructure which resumes the stream.
Using the Transforms
Example 1: globally default use
PCollection<Integer> input = ...;
PCollection<Long> hllSketch = input.apply(ApproximateDistinct.<Integer>globally());
Example 2: per key default use
PCollection<Integer, String> input = ...;
PCollection<Integer, Long> hllSketches = input.apply(ApproximateDistinct
.<Integer, String>perKey());
Example 3: tune precision and use sparse representation
One can tune the precision and sparse precision parameters in order to control the accuracy
and the memory. The tuning works exactly the same for globally() and perKey().
int precision = 15;
int sparsePrecision = 25;
PCollection<Double> input = ...;
PCollection<Long> hllSketch = input.apply(ApproximateDistinct
.<Double>globally()
.withPrecision(precision)
.withSparsePrecision(sparsePrecision));
Using the ApproximateDistinct.ApproximateDistinctFn CombineFn
The CombineFn does the same thing as the transform but it can be used in cases where you want
to manipulate the HyperLogLogPlus sketch, for example if you want to store it in a
database to have a backup. It can also be used in stateful processing or in CombineFns.ComposedCombineFn.
Example 1: basic use
This example is not really interesting but show how you can properly create an ApproximateDistinct.ApproximateDistinctFn. One must always specify a coder using the ApproximateDistinct.ApproximateDistinctFn.create(Coder) method.
PCollection<Integer> input = ...;
PCollection<HyperLogLogPlus> output = input.apply(Combine.globally(ApproximateDistinctFn
.<Integer>create(BigEndianIntegerCoder.of()));
Example 2: use the Combine.CombineFn in a stateful ParDo
One may want to use the ApproximateDistinct.ApproximateDistinctFn in a stateful ParDo in order to make
some processing depending on the current cardinality of the stream.
For more information about stateful processing see the blog spot on this topic here.
Here is an example of DoFn using an ApproximateDistinct.ApproximateDistinctFn as a CombiningState:
class StatefulCardinality<V> extends DoFn<V, OutputT> {
@StateId("hyperloglog")
private final StateSpec<CombiningState<V, HyperLogLogPlus, HyperLogLogPlus>>
indexSpec;
public StatefulCardinality(ApproximateDistinctFn<V> fn) {
indexSpec = StateSpecs.combining(fn);
}
@ProcessElement
public void processElement(
ProcessContext context,
@StateId("hllSketch")
CombiningState<V, HyperLogLogPlus, HyperLogLogPlus> hllSketch) {
long current = MoreObjects.firstNonNull(hllSketch.getAccum().cardinality(), 0L);
hllSketch.add(context.element());
context.output(...);
}
}
Then the DoFn can be called like this:
PCollection<V> input = ...;
ApproximateDistinctFn<V> myFn = ApproximateDistinctFn.create(input.getCoder());
PCollection<V> = input.apply(ParDo.of(new StatefulCardinality<>(myFn)));
Example 3: use the ApproximateDistinct.RetrieveCardinality utility class
One may want to retrieve the cardinality as a long after making some advanced processing using
the HyperLogLogPlus structure.
The ApproximateDistinct.RetrieveCardinality utility class provides an easy way to do so:
PCollection<MyObject> input = ...;
PCollection<HyperLogLogPlus> hll = input.apply(Combine.globally(ApproximateDistinctFn
.<MyObject>create(new MyObjectCoder())
.withSparseRepresentation(20)));
// Some advanced processing
PCollection<SomeObject> advancedResult = hll.apply(...);
PCollection<Long> cardinality = hll.apply(ApproximateDistinct.RetrieveCardinality.globally());
Consider using the HllCount.Init transform in the zetasketch extension module if
you need to create sketches compatible with Google Cloud BigQuery. For more details about using
HllCount and the zetasketch extension module, see
https://s.apache.org/hll-in-beam#bookmark=id.v6chsij1ixo7-
Nested Class Summary
Nested ClassesModifier and TypeClassDescriptionstatic classImplements theCombine.CombineFnofApproximateDistincttransforms.static classImplementation ofglobally().static classCoder forHyperLogLogPlusclass.static classImplementation ofperKey(). -
Constructor Summary
Constructors -
Method Summary
Modifier and TypeMethodDescriptionstatic <InputT> ApproximateDistinct.GloballyDistinct<InputT> globally()Computes the approximate number of distinct elements in the inputPCollection<InputT>and returns aPCollection<Long>.static <K,V> ApproximateDistinct.PerKeyDistinct <K, V> perKey()Likeglobally()but per key, i.e computes the approximate number of distinct values per key in aPCollection<KV<K, V>>and returnsPCollection<KV<K, Long>>.static longprecisionForRelativeError(double relativeError) Computes the precision based on the desired relative error.static doublerelativeErrorForPrecision(int p)
-
Constructor Details
-
ApproximateDistinct
public ApproximateDistinct()
-
-
Method Details
-
globally
Computes the approximate number of distinct elements in the inputPCollection<InputT>and returns aPCollection<Long>.- Type Parameters:
InputT- the type of the elements in the inputPCollection
-
perKey
Likeglobally()but per key, i.e computes the approximate number of distinct values per key in aPCollection<KV<K, V>>and returnsPCollection<KV<K, Long>>.- Type Parameters:
K- type of the keys mapping the elementsV- type of the values being combined per key
-
precisionForRelativeError
public static long precisionForRelativeError(double relativeError) Computes the precision based on the desired relative error.According to the paper, the mean squared error is bounded by the following formula:
b(m) / sqrt(m) Where m is the number of buckets used (
p = log2(m)) andb(m) < 1.106form > 16 (and p > 4).
WARNING:
This does not mean relative error in the estimation can't be higher.
This only means that on average the relative error will be lower than the desired relative error.
Nevertheless, the more elements arrive in thePCollection, the lower the variation will be.
Indeed, this is like when you throw a dice millions of time: the relative frequency of each different result{1,2,3,4,5,6}will get closer to1/6.- Parameters:
relativeError- the mean squared error should be in the interval ]0,1]- Returns:
- the minimum precision p in order to have the desired relative error on average.
-
relativeErrorForPrecision
public static double relativeErrorForPrecision(int p) - Parameters:
p- the precision i.e. the number of bits used for indexing the buckets- Returns:
- the Mean squared error of the Estimation of cardinality to expect for the given value of p.
-