[ 
https://issues.apache.org/jira/browse/BEAM-7013?focusedWorklogId=295533&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-295533
 ]

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_r314284036
 
 

 ##########
 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
+   * 
href="https://cloud.google.com/bigquery/docs/reference/standard-sql/hll_functions";>BigQuery</a>.
+   */
+  public static final class Init {
+
+    // Cannot be instantiated. This class is intended to be a namespace only.
+    private Init() {}
+
+    /**
+     * Returns a {@link Builder} for a {@code HllCount.Init} combining {@code 
PTransform} that
+     * computes integer-type HLL++ sketches. Call {@link Builder#globally()} 
or {@link
+     * Builder#perKey()} on the returning {@link Builder} to finalize the 
{@code PTransform}.
+     *
+     * <p>Calling {@link Builder#globally()} returns a {@code PTransform} that 
takes an input {@code
+     * PCollection<Integer>} and returns a {@code PCollection<byte[]>} whose 
contents is the
+     * integer-type HLL++ sketch computed from the elements in the input 
{@code PCollection}.
+     *
+     * <p>Calling {@link Builder#perKey()} returns a {@code PTransform} that 
takes an input {@code
+     * PCollection<KV<K, Integer>>} and returns a {@code PCollection<KV<K, 
byte[]>>} whose contents
+     * is the per-key integer-type HLL++ sketch computed from the values 
matching each key in the
+     * input {@code PCollection}.
+     *
+     * <p>Integer-type sketches cannot be merged with sketches of other types.
+     */
+    public static Builder<Integer> forIntegers() {
+      return new Builder<>(HllCountInitFn.forInteger());
+    }
+
+    /**
+     * Returns a {@link Builder} for a {@code HllCount.Init} combining {@code 
PTransform} that
+     * computes long-type HLL++ sketches. Call {@link Builder#globally()} or 
{@link
+     * Builder#perKey()} on the returning {@link Builder} to finalize the 
{@code PTransform}.
+     *
+     * <p>Calling {@link Builder#globally()} returns a {@code PTransform} that 
takes an input {@code
+     * PCollection<Long>} and returns a {@code PCollection<byte[]>} whose 
contents is the long-type
+     * HLL++ sketch computed from the elements in the input {@code 
PCollection}.
+     *
+     * <p>Calling {@link Builder#perKey()} returns a {@code PTransform} that 
takes an input {@code
+     * PCollection<KV<K, Long>>} and returns a {@code PCollection<KV<K, 
byte[]>>} whose contents is
+     * the per-key long-type HLL++ sketch computed from the values matching 
each key in the input
+     * {@code PCollection}.
+     *
+     * <p>Long-type sketches cannot be merged with sketches of other types.
+     */
+    public static Builder<Long> forLongs() {
+      return new Builder<>(HllCountInitFn.forLong());
+    }
+
+    /**
+     * Returns a {@link Builder} for a {@code HllCount.Init} combining {@code 
PTransform} that
+     * computes string-type HLL++ sketches. Call {@link Builder#globally()} or 
{@link
+     * Builder#perKey()} on the returning {@link Builder} to finalize the 
{@code PTransform}.
+     *
+     * <p>Calling {@link Builder#globally()} returns a {@code PTransform} that 
takes an input {@code
+     * PCollection<String>} and returns a {@code PCollection<byte[]>} whose 
contents is the
+     * string-type HLL++ sketch computed from the elements in the input {@code 
PCollection}.
+     *
+     * <p>Calling {@link Builder#perKey()} returns a {@code PTransform} that 
takes an input {@code
+     * PCollection<KV<K, String>>} and returns a {@code PCollection<KV<K, 
byte[]>>} whose contents
+     * is the per-key string-type HLL++ sketch computed from the values 
matching each key in the
+     * input {@code PCollection}.
+     *
+     * <p>String-type sketches cannot be merged with sketches of other types.
+     */
+    public static Builder<String> forStrings() {
+      return new Builder<>(HllCountInitFn.forString());
+    }
+
+    /**
+     * Returns a {@link Builder} for a {@code HllCount.Init} combining {@code 
PTransform} that
+     * computes bytes-type HLL++ sketches. Call {@link Builder#globally()} or 
{@link
+     * Builder#perKey()} on the returning {@link Builder} to finalize the 
{@code PTransform}.
+     *
+     * <p>Calling {@link Builder#globally()} returns a {@code PTransform} that 
takes an input {@code
+     * PCollection<byte[]>} and returns a {@code PCollection<byte[]>} whose 
contents is the
+     * bytes-type HLL++ sketch computed from the elements in the input {@code 
PCollection}.
+     *
+     * <p>Calling {@link Builder#perKey()} returns a {@code PTransform} that 
takes an input {@code
+     * PCollection<KV<K, byte[]>>} and returns a {@code PCollection<KV<K, 
byte[]>>} whose contents
+     * is the per-key bytes-type HLL++ sketch computed from the values 
matching each key in the
+     * input {@code PCollection}.
+     *
+     * <p>Bytes-type sketches cannot be merged with sketches of other types.
+     */
+    public static Builder<byte[]> forBytes() {
+      return new Builder<>(HllCountInitFn.forBytes());
+    }
+
+    /**
+     * Builder for the {@code HllCount.Init} combining {@code PTransform}.
+     *
+     * <p>Call {@link #withPrecision(int)} to customize the {@code precision} 
parameter of the
+     * sketch.
+     *
+     * <p>Call {@link #globally()} or {@link #perKey()} to finalize the {@code 
PTransform}.
+     *
+     * @param <InputT> element type or value type in {@code KV}s of the input 
{@code PCollection} to
+     *     the {@code PTransform} being built
+     */
+    public static final class Builder<InputT> {
+
+      private final HllCountInitFn<InputT, ?> initFn;
+
+      private Builder(HllCountInitFn<InputT, ?> initFn) {
+        this.initFn = initFn;
+      }
+
+      /**
+       * Explicitly set the {@code precision} parameter used to compute HLL++ 
sketch.
+       *
+       * <p>Valid range is between {@link #MINIMUM_PRECISION} and {@link 
#MAXIMUM_PRECISION}. If
+       * this method is not called, {@link #DEFAULT_PRECISION} will be used. 
Sketches computed using
+       * different {@code precision}s cannot be merged together.
+       *
+       * @param precision the {@code precision} parameter used to compute 
HLL++ sketch
+       */
+      public Builder<InputT> withPrecision(int precision) {
+        initFn.setPrecision(precision);
+        return this;
+      }
+
+      /**
+       * Returns a {@link Combine.Globally} {@code PTransform} that takes an 
input {@code
+       * PCollection<InputT>} and returns a {@code PCollection<byte[]>} whose 
contents is the HLL++
+       * sketch computed from the elements in the input {@code PCollection}.
+       *
+       * <p>Returns an empty output {@code PCollection} if the input {@code 
PCollection} is empty.
+       */
+      public Combine.Globally<InputT, byte[]> globally() {
+        return Combine.globally(initFn).withoutDefaults();
+      }
+
+      /**
+       * Returns a {@link Combine.PerKey} {@code PTransform} that takes an 
input {@code
+       * PCollection<KV<K, InputT>>} and returns a {@code PCollection<KV<K, 
byte[]>>} whose contents
+       * is the per-key HLL++ sketch computed from the values matching each 
key in the input {@code
+       * PCollection}.
+       */
+      public <K> Combine.PerKey<K, InputT, byte[]> perKey() {
+        return Combine.perKey(initFn);
+      }
+    }
+  }
+
+  /**
+   * Provide {@code PTransform}s to merge HLL++ sketches into a new sketch.
+   *
+   * <p>Only sketches of the same type and {@code precision} can be merged 
together. If incompatible
+   * sketches are provided, a runtime error will occur.
+   *
+   * <p>Correspond to the {@code HLL_COUNT.MERGE_PARTIAL(sketch)} function in 
<a
+   * 
href="https://cloud.google.com/bigquery/docs/reference/standard-sql/hll_functions";>BigQuery</a>.
+   */
+  public static final class MergePartial {
+
+    // Cannot be instantiated. This class is intended to be a namespace only.
+    private MergePartial() {}
+
+    /**
+     * Returns a {@link Combine.Globally} {@code PTransform} that takes an 
input {@code
+     * PCollection<byte[]>} of HLL++ sketches and returns a {@code 
PCollection<byte[]>} of a new
+     * sketch merged from the input sketches.
+     *
+     * <p>Only sketches of the same type and {@code precision} can be merged 
together. If
 
 Review comment:
   True for type; for precision, we can merge sketches of unequal precision; 
all sketches just get downgraded to the minimum precision encountered (see 
https://github.com/google/zetasketch/blob/master/java/com/google/zetasketch/internal/hllplus/Representation.java
 => merge(..)). 
   (Same for perKey(..) doc below)
   
 
----------------------------------------------------------------
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: 295533)
    Time Spent: 18h 20m  (was: 18h 10m)

> 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 20m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to