public final class HllCount
extends java.lang.Object
PTransform
s to compute HyperLogLogPlusPlus (HLL++) sketches on data streams based on the
ZetaSketch implementation.
HLL++ is an algorithm implemented by Google that estimates the count of distinct elements in a data stream. HLL++ requires significantly less memory than the linear memory needed for exact computation, at the cost of a small error. Cardinalities of arbitrary breakdowns can be computed using the HLL++ sketch. See this published paper for details about the algorithm.
HLL++ functions are also supported in Google Cloud
BigQuery. The HllCount PTransform
s provided here produce and consume sketches
compatible with BigQuery.
For detailed design of this class, see https://s.apache.org/hll-in-beam.
PCollection<Long>
and specify precision
PCollection<Long> input = ...;
int p = ...;
PCollection<byte[]> sketch = input.apply(HllCount.Init.forLongs().withPrecision(p).globally());
PCollection<KV<String, byte[]>>
PCollection<KV<String, byte[]>> input = ...;
PCollection<KV<String, byte[]>> sketch = input.apply(HllCount.Init.forBytes().perKey());
PCollection<byte[]>
into a new one
PCollection<byte[]> sketches = ...;
PCollection<byte[]> mergedSketch = sketches.apply(HllCount.MergePartial.globally());
PCollection<String>
PCollection<String> input = ...;
PCollection<Long> countDistinct =
input.apply(HllCount.Init.forStrings().globally()).apply(HllCount.Extract.globally());
Note: Currently HllCount does not work on FnAPI workers. See Issue #19698.Modifier and Type | Class and Description |
---|---|
static class |
HllCount.Extract
Provides
PTransform s to extract the estimated count of distinct elements (as Long s) from each HLL++ sketch. |
static class |
HllCount.Init
Provides
PTransform s to aggregate inputs into HLL++ sketches. |
static class |
HllCount.MergePartial
Provides
PTransform s to merge HLL++ sketches into a new sketch. |
Modifier and Type | Field and Description |
---|---|
static int |
DEFAULT_PRECISION
The default
precision value used in HllCount.Init.Builder.withPrecision(int) is
15. |
static int |
MAXIMUM_PRECISION
The maximum
precision value you can set in HllCount.Init.Builder.withPrecision(int) is
24. |
static int |
MINIMUM_PRECISION
The minimum
precision value you can set in HllCount.Init.Builder.withPrecision(int) is
10. |
Modifier and Type | Method and Description |
---|---|
static byte[] |
getSketchFromByteBuffer(@Nullable java.nio.ByteBuffer bf)
Converts the passed-in sketch from
ByteBuffer to byte[] , mapping null
ByteBuffer s (representing empty sketches) to empty byte[] s. |
public static final int MINIMUM_PRECISION
precision
value you can set in HllCount.Init.Builder.withPrecision(int)
is
10.public static final int MAXIMUM_PRECISION
precision
value you can set in HllCount.Init.Builder.withPrecision(int)
is
24.public static final int DEFAULT_PRECISION
precision
value used in HllCount.Init.Builder.withPrecision(int)
is
15.public static byte[] getSketchFromByteBuffer(@Nullable java.nio.ByteBuffer bf)
ByteBuffer
to byte[]
, mapping null
ByteBuffer
s (representing empty sketches) to empty byte[]
s.
Utility method to convert sketches materialized with ZetaSQL/BigQuery to valid inputs for
Beam HllCount
transforms.