Class HllCount
java.lang.Object
org.apache.beam.sdk.extensions.zetasketch.HllCount
PTransforms 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 PTransforms provided here produce and consume sketches
compatible with BigQuery.
For detailed design of this class, see https://s.apache.org/hll-in-beam.
Examples
Example 1: Create long-type sketch for a PCollection<Long> and specify precision
PCollection<Long> input = ...;
int p = ...;
PCollection<byte[]> sketch = input.apply(HllCount.Init.forLongs().withPrecision(p).globally());
Example 2: Create bytes-type sketch for a PCollection<KV<String, byte[]>>
PCollection<KV<String, byte[]>> input = ...;
PCollection<KV<String, byte[]>> sketch = input.apply(HllCount.Init.forBytes().perKey());
Example 3: Merge existing sketches in a PCollection<byte[]> into a new one
PCollection<byte[]> sketches = ...;
PCollection<byte[]> mergedSketch = sketches.apply(HllCount.MergePartial.globally());
Example 4: Estimates the count of distinct elements in a 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.-
Nested Class Summary
Nested ClassesModifier and TypeClassDescriptionstatic final classProvidesPTransforms to extract the estimated count of distinct elements (asLongs) from each HLL++ sketch.static final classProvidesPTransforms to aggregate inputs into HLL++ sketches.static final classProvidesPTransforms to merge HLL++ sketches into a new sketch. -
Field Summary
FieldsModifier and TypeFieldDescriptionstatic final intThe defaultprecisionvalue used inHllCount.Init.Builder.withPrecision(int)is 15.static final intThe maximumprecisionvalue you can set inHllCount.Init.Builder.withPrecision(int)is 24.static final intThe minimumprecisionvalue you can set inHllCount.Init.Builder.withPrecision(int)is 10. -
Method Summary
Modifier and TypeMethodDescriptionstatic byte[]Converts the passed-in sketch fromByteBuffertobyte[], mappingnull ByteBuffers (representing empty sketches) to emptybyte[]s.
-
Field Details
-
MINIMUM_PRECISION
public static final int MINIMUM_PRECISIONThe minimumprecisionvalue you can set inHllCount.Init.Builder.withPrecision(int)is 10.- See Also:
-
MAXIMUM_PRECISION
public static final int MAXIMUM_PRECISIONThe maximumprecisionvalue you can set inHllCount.Init.Builder.withPrecision(int)is 24.- See Also:
-
DEFAULT_PRECISION
public static final int DEFAULT_PRECISIONThe defaultprecisionvalue used inHllCount.Init.Builder.withPrecision(int)is 15.- See Also:
-
-
Method Details
-
getSketchFromByteBuffer
Converts the passed-in sketch fromByteBuffertobyte[], mappingnull ByteBuffers (representing empty sketches) to emptybyte[]s.Utility method to convert sketches materialized with ZetaSQL/BigQuery to valid inputs for Beam
HllCounttransforms.
-