[ https://issues.apache.org/jira/browse/BEAM-7013?focusedWorklogId=295531&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-295531 ]
ASF GitHub Bot logged work on BEAM-7013: ---------------------------------------- Author: ASF GitHub Bot Created on: 15/Aug/19 16:33 Start Date: 15/Aug/19 16:33 Worklog Time Spent: 10m Work Description: zfraa commented on pull request #9144: [BEAM-7013] Integrating ZetaSketch's HLL++ algorithm with Beam URL: https://github.com/apache/beam/pull/9144#discussion_r313900745 ########## File path: sdks/java/extensions/zetasketch/src/main/java/org/apache/beam/sdk/extensions/zetasketch/HllCount.java ########## @@ -0,0 +1,364 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.zetasketch; + +import com.google.zetasketch.HyperLogLogPlusPlus; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +/** + * {@code PTransform}s to compute HyperLogLogPlusPlus (HLL++) sketches on data streams based on the + * <a href="https://github.com/google/zetasketch">ZetaSketch</a> implementation. + * + * <p>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 <a + * href="http://static.googleusercontent.com/media/research.google.com/en/us/pubs/archive/40671.pdf">published + * paper</a> for details about the algorithm. + * + * <p>HLL++ functions are also supported in <a + * href="https://cloud.google.com/bigquery/docs/reference/standard-sql/hll_functions">Google Cloud + * BigQuery</a>. The {@code HllCount PTransform}s provided here produce and consume sketches + * compatible with BigQuery. + * + * <p>For detailed design of this class, see https://s.apache.org/hll-in-beam. + * + * <h3>Examples</h3> + * + * <h4>Example 1: Create long-type sketch for a {@code PCollection<Long>} and specify precision</h4> + * + * <pre>{@code + * PCollection<Long> input = ...; + * int p = ...; + * PCollection<byte[]> sketch = input.apply(HllCount.Init.forLongs().withPrecision(p).globally()); + * }</pre> + * + * <h4>Example 2: Create bytes-type sketch for a {@code PCollection<KV<String, byte[]>>}</h4> + * + * <pre>{@code + * PCollection<KV<String, byte[]>> input = ...; + * PCollection<KV<String, byte[]>> sketch = input.apply(HllCount.Init.forBytes().perKey()); + * }</pre> + * + * <h4>Example 3: Merge existing sketches in a {@code PCollection<byte[]>} into a new one</h4> + * + * <pre>{@code + * PCollection<byte[]> sketches = ...; + * PCollection<byte[]> mergedSketch = sketches.apply(HllCount.MergePartial.globally()); + * }</pre> + * + * <h4>Example 4: Estimates the count of distinct elements in a {@code PCollection<String>}</h4> + * + * <pre>{@code + * PCollection<String> input = ...; + * PCollection<Long> countDistinct = + * input.apply(HllCount.Init.forStrings().globally()).apply(HllCount.Extract.globally()); + * }</pre> + * + * Note: Currently HllCount does not work on FnAPI workers. See <a + * href="https://issues.apache.org/jira/browse/BEAM-7879">Jira ticket [BEAM-7879]</a>. + */ +@Experimental +public final class HllCount { + + /** + * The minimum {@code precision} value you can set in {@link Init.Builder#withPrecision(int)} is + * {@value}. + */ + public static final int MINIMUM_PRECISION = HyperLogLogPlusPlus.MINIMUM_PRECISION; + + /** + * The maximum {@code precision} value you can set in {@link Init.Builder#withPrecision(int)} is + * {@value}. + */ + public static final int MAXIMUM_PRECISION = HyperLogLogPlusPlus.MAXIMUM_PRECISION; + + /** + * The default {@code precision} value used in {@link Init.Builder#withPrecision(int)} is + * {@value}. + */ + public static final int DEFAULT_PRECISION = HyperLogLogPlusPlus.DEFAULT_NORMAL_PRECISION; + + // Cannot be instantiated. This class is intended to be a namespace only. + private HllCount() {} + + /** + * Provide {@code PTransform}s to aggregate inputs into HLL++ sketches. The four supported input + * types are {@code Integer}, {@code Long}, {@code String}, and {@code byte[]}. + * + * <p>Sketches are represented using the {@code byte[]} type. Sketches of the same type and {@code + * precision} can be merged into a new sketch using {@link HllCount.MergePartial}. Estimated count + * of distinct elements can be extracted from sketches using {@link HllCount.Extract}. + * + * <p>Correspond to the {@code HLL_COUNT.INIT(input [, precision])} function in <a Review comment: nit: s/Correspond/Corresponds ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 295531) > A new count distinct transform based on BigQuery compatible HyperLogLog++ > implementation > ---------------------------------------------------------------------------------------- > > Key: BEAM-7013 > URL: https://issues.apache.org/jira/browse/BEAM-7013 > Project: Beam > Issue Type: New Feature > Components: extensions-java-sketching, sdk-java-core > Reporter: Yueyang Qiu > Assignee: Yueyang Qiu > Priority: Major > Fix For: 2.16.0 > > Time Spent: 18h > Remaining Estimate: 0h > -- This message was sent by Atlassian JIRA (v7.6.14#76016)